You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by jb...@apache.org on 2016/06/23 14:15:49 UTC

[01/56] [abbrv] incubator-carbondata git commit: [Issue 618]Supported Spark 1.6 in Carbondata (#670)

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master [created] 63d3284e4


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
index b19b744..4edfb7a 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
@@ -40,11 +40,14 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll {
     CarbonProperties.getInstance().addProperty("carbon.direct.surrogate","false")
     sql("create cube Carbon_automation_test dimensions(imei string,deviceInformationId integer,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active_
 phonePADPartitionedVersions string, Latest_YEAR integer, Latest_MONTH integer, Latest_DAY integer, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string)  measures(gamePointId integer,contractNumber integer) OPTIONS (PARTITIONER [CLASS = 'org.carbondata.spark.partition.api.impl.SampleDataPartitionerImpl' ,COLUMNS= (imei) , PARTITION_COUNT=2] )");
     sql("LOAD DATA FACT FROM'"+currentDirectory+"/src/test/resources/100_olap.csv' INTO Cube Carbon_automation_test partitionData(DELIMITER ',' ,QUOTECHAR '\"', FILEHEADER 'imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUIV
 ersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')");
+    sql("create table if not exists Carbon_automation_hive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active
 _phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber int) row format delimited fields terminated by ','");
+    sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO table Carbon_automation_hive ");
 
   }
 
   override def afterAll {
     sql("drop cube Carbon_automation_test")
+    sql("drop cube Carbon_automation_hive")
 
   }
 
@@ -414,7 +417,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll {
   test("select variance(deviceInformationId) as a   from Carbon_automation_test")({
     checkAnswer(
       sql("select variance(deviceInformationId) as a   from Carbon_automation_test"),
-      Seq(Row(9.31041555963636E9)))
+      sql("select variance(deviceInformationId) as a   from Carbon_automation_hive"))
   })
    //TC_105
   test("select var_samp(deviceInformationId) as a  from Carbon_automation_test")({
@@ -455,7 +458,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll {
   test("select corr(deviceInformationId,deviceInformationId)  as a from Carbon_automation_test")({
     checkAnswer(
       sql("select corr(deviceInformationId,deviceInformationId)  as a from Carbon_automation_test"),
-      Seq(Row(1.0000000000000002)))
+      sql("select corr(deviceInformationId,deviceInformationId)  as a from Carbon_automation_hive"))
   })
 
   //TC_111

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
index 061b265..4012493 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala
@@ -18,11 +18,22 @@ import org.scalatest.BeforeAndAfterAll
   */
 class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
 
+  // return segment details
+  def getSegments(databaseName : String, tableName : String, tableId : String): List[String] = {
+    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(new
+        AbsoluteTableIdentifier(
+          CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
+          new CarbonTableIdentifier(databaseName, tableName.toLowerCase , tableId)
+        )
+    )
+    val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+    segments
+  }
+
   override def beforeAll {
-    sql("drop table if exists  noDictionaryCompaction")
 
     sql(
-      "CREATE TABLE IF NOT EXISTS noDictionaryCompaction (country String, ID Int, date Timestamp, name " +
+      "CREATE TABLE nodictionaryCompaction (country String, ID Int, date Timestamp, name " +
         "String, " +
         "phonetype String, serialname String, salary Int) STORED BY 'org.apache.carbondata" +
         ".format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='country')"
@@ -37,17 +48,17 @@ class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/mm/dd")
-    sql("LOAD DATA fact from '" + csvFilePath1 + "' INTO CUBE noDictionaryCompaction PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath1 + "' INTO TABLE nodictionaryCompaction " +
+        "OPTIONS('DELIMITER' = ',')"
     )
-    sql("LOAD DATA fact from '" + csvFilePath2 + "' INTO CUBE noDictionaryCompaction  PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath2 + "' INTO TABLE nodictionaryCompaction " +
+        "OPTIONS('DELIMITER' = ',')"
     )
-    sql("LOAD DATA fact from '" + csvFilePath3 + "' INTO CUBE noDictionaryCompaction  PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath3 + "' INTO TABLE nodictionaryCompaction " +
+        "OPTIONS('DELIMITER' = ',')"
     )
     // compaction will happen here.
-    sql("alter table noDictionaryCompaction compact 'major'"
+    sql("alter table nodictionaryCompaction compact 'major'"
     )
 
     // wait for compaction to finish.
@@ -62,13 +73,7 @@ class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
     var noOfRetries = 0
     while (status && noOfRetries < 10) {
 
-      val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(new
-          AbsoluteTableIdentifier(
-            CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
-            new CarbonTableIdentifier("default", "noDictionaryCompaction", "1")
-          )
-      )
-      val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+      val segments: List[String] = getSegments("default", "nodictionaryCompaction", "uni21")
 
       if (!segments.contains("0.1")) {
         // wait for 2 seconds for compaction to complete.
@@ -81,11 +86,10 @@ class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
     }
   }
 
-
-  test("select country from noDictionaryCompaction") {
+  test("select country from nodictionaryCompaction") {
     // check answers after compaction.
     checkAnswer(
-      sql("select country from noDictionaryCompaction"),
+      sql("select country from nodictionaryCompaction"),
       Seq(Row("america"),
         Row("canada"),
         Row("chile"),
@@ -107,16 +111,10 @@ class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
 
   test("delete merged folder and execute query") {
     // delete merged segments
-   sql("clean files for table noDictionaryCompaction")
+   sql("clean files for table nodictionaryCompaction")
 
-    val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(new
-        AbsoluteTableIdentifier(
-          CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
-          new CarbonTableIdentifier("default", "noDictionaryCompaction", "1")
-        )
-    )
     // merged segment should not be there
-    val segments   = segmentStatusManager.getValidSegments.listOfValidSegments.asScala.toList
+    val segments = getSegments("default", "nodictionaryCompaction", "uni21")
     assert(!segments.contains("0"))
     assert(!segments.contains("1"))
     assert(!segments.contains("2"))
@@ -124,7 +122,7 @@ class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
 
     // now check the answers it should be same.
     checkAnswer(
-      sql("select country from noDictionaryCompaction"),
+      sql("select country from nodictionaryCompaction"),
       Seq(Row("america"),
         Row("canada"),
         Row("chile"),
@@ -145,7 +143,7 @@ class DataCompactionNoDictionaryTest extends QueryTest with BeforeAndAfterAll {
   }
 
   override def afterAll {
-    sql("drop cube noDictionaryCompaction")
+    sql("drop table nodictionaryCompaction")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
         CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
index 7fb72f5..5b473e2 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataretention/DataRetentionTestCase.scala
@@ -96,7 +96,7 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
 
   test("RetentionTest_withoutDelete") {
     checkAnswer(
-      sql("SELECT country, count(salary) AS amount FROM DataRetentionTable WHERE country" +
+      sql("SELECT country, count(salary) AS amount FROM dataretentionTable WHERE country" +
           " IN ('china','ind','aus','eng') GROUP BY country"
       ),
       Seq(Row("aus", 9), Row("ind", 9))
@@ -114,7 +114,7 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
     val actualValue: String = getSegmentStartTime(segments, 1)
     // delete segments (0,1) which contains ind, aus
     sql(
-      "DELETE SEGMENTS FROM TABLE DataRetentionTable where STARTTIME before '" + actualValue + "'")
+      "DELETE SEGMENTS FROM TABLE dataretentionTable where STARTTIME before '" + actualValue + "'")
 
     // load segment 2 which contains eng
     sql(
@@ -130,9 +130,9 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
 
   test("RetentionTest3_DeleteByLoadId") {
     // delete segment 2 and load ind segment
-    sql("DELETE LOAD 2 FROM TABLE DataRetentionTable")
+    sql("DELETE LOAD 2 FROM TABLE dataretentionTable")
     sql(
-      "LOAD DATA LOCAL INPATH '" + resource + "dataretention1.csv' INTO TABLE DataRetentionTable " +
+      "LOAD DATA LOCAL INPATH '" + resource + "dataretention1.csv' INTO TABLE dataretentionTable " +
       "OPTIONS('DELIMITER' = ',')")
     checkAnswer(
       sql("SELECT country, count(salary) AS amount FROM DataRetentionTable WHERE country" +
@@ -140,7 +140,10 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
       ),
       Seq(Row("ind", 9))
     )
-    sql("clean files for table DataRetentionTable")
+
+    // these queries should execute without any error.
+    sql("show segments for table dataretentionTable")
+    sql("clean files for table dataretentionTable")
   }
 
   test("RetentionTest4_DeleteByInvalidLoadId") {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
index dc89a93..aeac733 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/deleteTable/TestDeleteTableNewDDL.scala
@@ -53,8 +53,31 @@ class TestDeleteTableNewDDL extends QueryTest with BeforeAndAfterAll {
 
   }
 
+  test("drop table using case insensitive table name") {
+    // create table
+    sql(
+      "CREATE table CaseInsensitiveTable (ID int, date String, country String, name " +
+      "String," +
+      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'" +
+      "TBLPROPERTIES('DICTIONARY_INCLUDE'='ID', 'DICTIONARY_INCLUDE'='salary')"
+    )
+    // table should drop wihout any error
+    sql("drop table caseInsensitiveTable")
+
+    // Now create same table, it should not give any error.
+    sql(
+      "CREATE table CaseInsensitiveTable (ID int, date String, country String, name " +
+      "String," +
+      "phonetype String, serialname String, salary int) stored by 'org.apache.carbondata.format'" +
+      "TBLPROPERTIES('DICTIONARY_INCLUDE'='ID', 'DICTIONARY_INCLUDE'='salary')"
+    )
+
+  }
+
+
   override def afterAll: Unit = {
 
+    sql("drop table CaseSensitiveTable")
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala
index 31b36d1..506c0fe 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeDirectDictionaryTestCase.scala
@@ -97,6 +97,12 @@ class TimestampDataTypeDirectDictionaryTest extends QueryTest with BeforeAndAfte
 
   }
 
+  test("select count(doj) from directDictionaryCube") {
+    checkAnswer(
+      sql("select count(doj) from directDictionaryCube"),
+      Seq(Row(2))
+    )
+  }
 
   override def afterAll {
     sql("drop cube directDictionaryCube")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
index fd8bb7c..ef59484 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/util/AutoHighCardinalityIdentifyTestCase.scala
@@ -108,7 +108,7 @@ class AutoHighCardinalityIdentifyTestCase extends QueryTest with BeforeAndAfterA
 
   def relation: CarbonRelation = {
     CarbonEnv.getInstance(CarbonHiveContext).carbonCatalog
-        .lookupRelation1(Option("default"), "highcard", None)(CarbonHiveContext)
+        .lookupRelation1(Option("default"), "highcard")(CarbonHiveContext)
         .asInstanceOf[CarbonRelation]
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
index 916d1f4..78a0eca 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
@@ -145,16 +145,16 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
 
   def buildRelation() = {
     val catalog = CarbonEnv.getInstance(CarbonHiveContext).carbonCatalog
-    sampleRelation = catalog.lookupRelation1(Option("default"), "sample", None)(CarbonHiveContext)
+    sampleRelation = catalog.lookupRelation1(Option("default"), "sample")(CarbonHiveContext)
       .asInstanceOf[CarbonRelation]
     dimSampleRelation = catalog
-      .lookupRelation1(Option("default"), "dimSample", None)(CarbonHiveContext)
+      .lookupRelation1(Option("default"), "dimSample")(CarbonHiveContext)
       .asInstanceOf[CarbonRelation]
     complexRelation = catalog
-      .lookupRelation1(Option("default"), "complextypes", None)(CarbonHiveContext)
+      .lookupRelation1(Option("default"), "complextypes")(CarbonHiveContext)
       .asInstanceOf[CarbonRelation]
     incrementalLoadTableRelation = catalog
-      .lookupRelation1(Option("default"), "incrementalLoadTable", None)(CarbonHiveContext)
+      .lookupRelation1(Option("default"), "incrementalLoadTable")(CarbonHiveContext)
       .asInstanceOf[CarbonRelation]
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 80d8c97..5df017a 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,7 +25,7 @@
   <name>carbondata</name>
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <spark.version>1.5.2</spark.version>
+    <spark.version>1.6.1</spark.version>
     <scala.binary.version>2.10</scala.binary.version>
     <snappy.version>1.1.1.7</snappy.version>
     <hadoop.version>2.2.0</hadoop.version>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/processing/pom.xml
----------------------------------------------------------------------
diff --git a/processing/pom.xml b/processing/pom.xml
index a8b1e6e..5cef13d 100644
--- a/processing/pom.xml
+++ b/processing/pom.xml
@@ -65,17 +65,6 @@
       <version>${kettle.version}</version>
     </dependency>
     <dependency>
-      <groupId>eigenbase</groupId>
-      <artifactId>eigenbase-xom</artifactId>
-      <version>1.3.4</version>
-      <exclusions>
-        <exclusion>
-          <groupId>*</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>commons-vfs</groupId>
       <artifactId>commons-vfs</artifactId>
       <version>1.0</version>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/processing/src/main/java/org/carbondata/processing/store/CarbonDataFileAttributes.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/CarbonDataFileAttributes.java b/processing/src/main/java/org/carbondata/processing/store/CarbonDataFileAttributes.java
index b17ed8d..e4f72b6 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonDataFileAttributes.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonDataFileAttributes.java
@@ -25,8 +25,6 @@ import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.constants.CarbonCommonConstants;
 
-import org.apache.spark.sql.columnar.TIMESTAMP;
-
 /**
  * This class contains attributes of file which are required to
  * construct file name like taskId, factTimeStamp
@@ -84,7 +82,7 @@ public class CarbonDataFileAttributes {
       dateToStr = parser.parse(factTimeStamp);
       return Long.toString(dateToStr.getTime());
     } catch (ParseException e) {
-      LOGGER.error("Cannot convert" + TIMESTAMP.toString()
+      LOGGER.error("Cannot convert" + factTimeStamp
           + " to Time/Long type value" + e.getMessage());
       return null;
     }


[56/56] [abbrv] incubator-carbondata git commit: Refactor carbon-core module for code clean up (#720)

Posted by jb...@apache.org.
Refactor carbon-core module for code clean up (#720)

* refactor core module

* fix style


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

Branch: refs/heads/master
Commit: 63d3284e4f0a09ef5248065da19c2bd3db371e08
Parents: d95742d
Author: Jacky Li <ja...@huawei.com>
Authored: Thu Jun 23 14:19:58 2016 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Thu Jun 23 11:49:58 2016 +0530

----------------------------------------------------------------------
 .../org/carbondata/common/CarbonIterator.java   |  38 ++
 .../store/columnar/ColumnGroupModel.java        | 137 +++++
 .../store/columnar/ColumnarKeyStoreInfo.java    |   1 -
 .../store/columnar/UnBlockIndexer.java          |  23 -
 .../fileperations/AtomicFileOperations.java     |  33 --
 .../fileperations/AtomicFileOperationsImpl.java |  87 ----
 .../store/fileperations/FileWriteOperation.java |  25 -
 .../core/datastorage/util/StoreFactory.java     | 143 +-----
 .../core/file/manager/composite/FileData.java   |  74 ---
 .../file/manager/composite/FileManager.java     |  68 ---
 .../composite/IFileManagerComposite.java        |  59 ---
 .../core/iterator/CarbonIterator.java           |  38 --
 .../core/locks/AbstractCarbonLock.java          |  77 ---
 .../core/locks/CarbonLockFactory.java           |  72 ---
 .../org/carbondata/core/locks/HdfsFileLock.java |  89 ----
 .../org/carbondata/core/locks/ICarbonLock.java  |  40 --
 .../carbondata/core/locks/LocalFileLock.java    | 162 ------
 .../org/carbondata/core/locks/LockUsage.java    |  29 --
 .../carbondata/core/locks/ZooKeeperLocking.java | 171 -------
 .../carbondata/core/locks/ZookeeperInit.java    |  64 ---
 .../core/metadata/LevelNormalizedProps.java     | 107 ----
 .../carbondata/core/metadata/SliceMetaData.java | 333 -------------
 .../carbondata/core/util/CarbonMergerUtil.java  |  75 ---
 .../core/util/CarbonSliceAndFiles.java          |  88 ----
 .../org/carbondata/core/util/CarbonUtil.java    | 499 +------------------
 .../carbondata/core/vo/ColumnGroupModel.java    | 137 -----
 .../core/writer/CarbonDataWriter.java           | 482 ------------------
 .../carbondata/scan/executor/QueryExecutor.java |   2 +-
 .../scan/executor/impl/DetailQueryExecutor.java |   2 +-
 .../impl/DetailRawRecordQueryExecutor.java      |   2 +-
 .../resolver/RestructureFilterResolverImpl.java |   8 -
 .../processor/AbstractDataBlockIterator.java    |   2 +-
 .../scan/processor/BlockletIterator.java        |   2 +-
 .../org/carbondata/scan/result/BatchResult.java |   2 +-
 .../AbstractDetailQueryResultIterator.java      |   2 +-
 .../scan/result/iterator/ChunkRowIterator.java  |   2 +-
 .../scan/result/iterator/RawResultIterator.java |   2 +-
 .../core/locks/LocalFileLockTest.java           |  48 --
 .../core/locks/ZooKeeperLockingTest.java        | 125 -----
 .../carbondata/hadoop/CarbonRecordReader.java   |   2 +-
 .../hadoop/test/util/StoreCreator.java          |   6 +-
 .../spark/merger/CarbonCompactionExecutor.java  |   2 +-
 .../spark/merger/RowResultMerger.java           |   2 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |   6 +-
 .../execution/command/carbonTableSchema.scala   |   3 +-
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |   6 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |   8 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   2 +-
 .../fileoperations/AtomicFileOperations.java    |  33 ++
 .../AtomicFileOperationsImpl.java               |  87 ++++
 .../lcm/fileoperations/FileWriteOperation.java  |  25 +
 .../lcm/locks/AbstractCarbonLock.java           |  77 +++
 .../carbondata/lcm/locks/CarbonLockFactory.java |  72 +++
 .../org/carbondata/lcm/locks/HdfsFileLock.java  |  89 ++++
 .../org/carbondata/lcm/locks/ICarbonLock.java   |  40 ++
 .../org/carbondata/lcm/locks/LocalFileLock.java | 162 ++++++
 .../org/carbondata/lcm/locks/LockUsage.java     |  29 ++
 .../carbondata/lcm/locks/ZooKeeperLocking.java  | 171 +++++++
 .../org/carbondata/lcm/locks/ZookeeperInit.java |  64 +++
 .../lcm/status/SegmentStatusManager.java        |  12 +-
 .../processing/mdkeygen/MDKeyGenStep.java       |   8 +-
 .../processing/mdkeygen/file/FileData.java      |  74 +++
 .../processing/mdkeygen/file/FileManager.java   |  68 +++
 .../mdkeygen/file/IFileManagerComposite.java    |  59 +++
 .../merger/util/CarbonSliceMergerUtil.java      | 366 --------------
 .../store/CarbonFactDataHandlerColumnar.java    |   6 +-
 .../store/CarbonFactDataHandlerModel.java       |   2 +-
 .../store/colgroup/ColGroupDataHolder.java      |   2 +-
 .../store/colgroup/ColGroupMinMax.java          |   2 +-
 .../store/writer/AbstractFactDataWriter.java    |   4 +-
 ...actDataWriterImplForIntIndexAndAggBlock.java |   2 +-
 .../CarbonCSVBasedDimSurrogateKeyGen.java       |   2 +-
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |   6 +-
 .../FileStoreSurrogateKeyGenForCSV.java         |   6 +-
 .../carbondata/lcm/locks/LocalFileLockTest.java |  48 ++
 .../lcm/locks/ZooKeeperLockingTest.java         | 125 +++++
 .../store/colgroup/ColGroupMinMaxTest.java      |   7 +-
 77 files changed, 1458 insertions(+), 3577 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/common/src/main/java/org/carbondata/common/CarbonIterator.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/carbondata/common/CarbonIterator.java b/common/src/main/java/org/carbondata/common/CarbonIterator.java
new file mode 100644
index 0000000..2ac2ff3
--- /dev/null
+++ b/common/src/main/java/org/carbondata/common/CarbonIterator.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.carbondata.common;
+
+import java.util.Iterator;
+
+/**
+ * CarbonIterator adds default implement for remove. This is required for Java 7.
+ * @param <E>
+ */
+public abstract class CarbonIterator<E> implements Iterator<E> {
+
+  @Override public abstract boolean hasNext();
+
+  @Override public abstract E next();
+
+  @Override public void remove() {
+    throw new UnsupportedOperationException("remove");
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnGroupModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnGroupModel.java b/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnGroupModel.java
new file mode 100644
index 0000000..532c6e9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnGroupModel.java
@@ -0,0 +1,137 @@
+/*
+ * 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.carbondata.core.datastorage.store.columnar;
+
+public class ColumnGroupModel {
+
+  /**
+   * cardinality all dimension
+   */
+  private int[] colGroupCardinality;
+
+  /**
+   * number of columns in columnar block
+   */
+  private int[] columnSplit;
+
+  /**
+   * total number of columns
+   */
+  private int noOfColumnsStore;
+
+  /**
+   * whether given index is columnar or not
+   * true: columnar
+   * false: row block
+   */
+  private boolean[] columnarStore;
+
+  /**
+   * column groups
+   * e.g
+   * {{0,1,2},3,4,{5,6}}
+   */
+  private int[][] columnGroups;
+
+  /**
+   * @return cardinality of column groups
+   */
+  public int[] getColumnGroupCardinality() {
+    return colGroupCardinality;
+  }
+
+  /**
+   * set columngroup cardinality
+   *
+   * @param columnGroupCardinality
+   */
+  public void setColumnGroupCardinality(int[] columnGroupCardinality) {
+    this.colGroupCardinality = columnGroupCardinality;
+  }
+
+  /**
+   * return columnSplit
+   *
+   * @return
+   */
+  public int[] getColumnSplit() {
+    return columnSplit;
+  }
+
+  /**
+   * set columnSplit
+   *
+   * @param split
+   */
+  public void setColumnSplit(int[] split) {
+    this.columnSplit = split;
+  }
+
+  /**
+   * @return no of columnar block
+   */
+  public int getNoOfColumnStore() {
+    return this.noOfColumnsStore;
+  }
+
+  /**
+   * set no of columnar block
+   *
+   * @param noOfColumnsStore
+   */
+  public void setNoOfColumnStore(int noOfColumnsStore) {
+    this.noOfColumnsStore = noOfColumnsStore;
+  }
+
+  /**
+   * it's an identifier for row block or single column block
+   *
+   * @param columnarStore
+   */
+  public void setColumnarStore(boolean[] columnarStore) {
+    this.columnarStore = columnarStore;
+  }
+
+  /**
+   * set column groups
+   *
+   * @param columnGroups
+   */
+  public void setColumnGroup(int[][] columnGroups) {
+    this.columnGroups = columnGroups;
+  }
+
+  /**
+   * check if given column group is columnar
+   *
+   * @param colGroup
+   * @return true if given block is columnar
+   */
+  public boolean isColumnar(int colGroup) {
+    return columnarStore[colGroup];
+  }
+
+  /**
+   * @return columngroups
+   */
+  public int[][] getColumnGroup() {
+    return this.columnGroups;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnarKeyStoreInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnarKeyStoreInfo.java b/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnarKeyStoreInfo.java
index c80a887..0cf0d99 100644
--- a/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnarKeyStoreInfo.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnarKeyStoreInfo.java
@@ -20,7 +20,6 @@
 package org.carbondata.core.datastorage.store.columnar;
 
 import org.carbondata.core.keygenerator.mdkey.NumberCompressor;
-import org.carbondata.core.vo.ColumnGroupModel;
 
 public class ColumnarKeyStoreInfo {
   private int numberOfKeys;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/datastorage/store/columnar/UnBlockIndexer.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/columnar/UnBlockIndexer.java b/core/src/main/java/org/carbondata/core/datastorage/store/columnar/UnBlockIndexer.java
index 26eb2ac..a20320b 100644
--- a/core/src/main/java/org/carbondata/core/datastorage/store/columnar/UnBlockIndexer.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/columnar/UnBlockIndexer.java
@@ -27,29 +27,6 @@ public final class UnBlockIndexer {
 
   }
 
-  public static short[] uncompressIndex(short[] indexData, short[] indexMap) {
-    int actualSize = indexData.length;
-    for (int i = 0; i < indexMap.length; i++) {
-      actualSize += indexData[indexMap[i] + 1] - indexData[indexMap[i]] - 1;
-    }
-    short[] indexes = new short[actualSize];
-    int k = 0;
-    for (short i = 0; i < indexData.length; i++) {
-      int index = Arrays.binarySearch(indexMap, i);
-      if (index > -1) {
-        for (short j = indexData[indexMap[index]]; j <= indexData[indexMap[index] + 1]; j++) {
-          indexes[k] = j;
-          k++;
-        }
-        i++;
-      } else {
-        indexes[k] = indexData[i];
-        k++;
-      }
-    }
-    return indexes;
-  }
-
   public static int[] uncompressIndex(int[] indexData, int[] indexMap) {
     int actualSize = indexData.length;
     for (int i = 0; i < indexMap.length; i++) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperations.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperations.java b/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperations.java
deleted file mode 100644
index c5dcb29..0000000
--- a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperations.java
+++ /dev/null
@@ -1,33 +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.carbondata.core.datastorage.store.fileperations;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-public interface AtomicFileOperations {
-
-  DataInputStream openForRead() throws IOException;
-
-  void close() throws IOException;
-
-  DataOutputStream openForWrite(FileWriteOperation operation) throws IOException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java b/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java
deleted file mode 100644
index 41f4580..0000000
--- a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java
+++ /dev/null
@@ -1,87 +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.carbondata.core.datastorage.store.fileperations;
-
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
-import java.io.IOException;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
-
-public class AtomicFileOperationsImpl implements AtomicFileOperations {
-
-  private String filePath;
-
-  private FileType fileType;
-
-  private String tempWriteFilePath;
-
-  private DataOutputStream dataOutStream;
-
-  public AtomicFileOperationsImpl(String filePath, FileType fileType) {
-    this.filePath = filePath;
-
-    this.fileType = fileType;
-  }
-
-  @Override public DataInputStream openForRead() throws IOException {
-    return FileFactory.getDataInputStream(filePath, fileType);
-  }
-
-  @Override public DataOutputStream openForWrite(FileWriteOperation operation) throws IOException {
-
-    filePath = filePath.replace("\\", "/");
-
-    tempWriteFilePath = filePath + CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
-
-    if (FileFactory.isFileExist(tempWriteFilePath, fileType)) {
-      FileFactory.getCarbonFile(tempWriteFilePath, fileType).delete();
-    }
-
-    FileFactory.createNewFile(tempWriteFilePath, fileType);
-
-    dataOutStream = FileFactory.getDataOutputStream(tempWriteFilePath, fileType);
-
-    return dataOutStream;
-
-  }
-
-  /* (non-Javadoc)
-   * @see com.huawei.unibi.carbon.datastorage.store.fileperations.AtomicFileOperations#close()
-   */
-  @Override public void close() throws IOException {
-
-    if (null != dataOutStream) {
-      dataOutStream.close();
-
-      CarbonFile tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
-
-      if (!tempFile.renameForce(filePath)) {
-        throw new IOException("temporary file renaming failed, src="
-            + tempFile.getPath() + ", dest=" + filePath);
-      }
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/FileWriteOperation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/FileWriteOperation.java b/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/FileWriteOperation.java
deleted file mode 100644
index 58026d9..0000000
--- a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/FileWriteOperation.java
+++ /dev/null
@@ -1,25 +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.carbondata.core.datastorage.store.fileperations;
-
-public enum FileWriteOperation {
-
-  APPEND, OVERWRITE
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java b/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
index 725ee56..8fc5aac 100644
--- a/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
@@ -20,63 +20,28 @@
 package org.carbondata.core.datastorage.util;
 
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.datastorage.store.NodeKeyStore;
 import org.carbondata.core.datastorage.store.NodeMeasureDataStore;
-import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStore;
-import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
-import org.carbondata.core.datastorage.store.impl.data.compressed.HeavyCompressedDoubleArrayDataFileStore;
 import org.carbondata.core.datastorage.store.impl.data.compressed.HeavyCompressedDoubleArrayDataInMemoryStore;
-import org.carbondata.core.datastorage.store.impl.data.uncompressed.DoubleArrayDataFileStore;
+
 import org.carbondata.core.datastorage.store.impl.data.uncompressed.DoubleArrayDataInMemoryStore;
-import org.carbondata.core.datastorage.store.impl.key.columnar.compressed.CompressedColumnarFileKeyStore;
-import org.carbondata.core.datastorage.store.impl.key.columnar.compressed.CompressedColumnarInMemoryStore;
-import org.carbondata.core.datastorage.store.impl.key.columnar.uncompressed.UnCompressedColumnarFileKeyStore;
-import org.carbondata.core.datastorage.store.impl.key.columnar.uncompressed.UnCompressedColumnarInMemoryStore;
-import org.carbondata.core.datastorage.store.impl.key.compressed.CompressedSingleArrayKeyFileStore;
-import org.carbondata.core.datastorage.store.impl.key.compressed.CompressedSingleArrayKeyInMemoryStore;
-import org.carbondata.core.datastorage.store.impl.key.uncompressed.SingleArrayKeyFileStore;
-import org.carbondata.core.datastorage.store.impl.key.uncompressed.SingleArrayKeyInMemoryStore;
 import org.carbondata.core.util.CarbonProperties;
 
 public final class StoreFactory {
   /**
-   * Single Array Key store.
-   */
-  private static final String SINGLE_ARRAY = "SINGLE_ARRAY";
-  /**
-   * Compressed single array key store.
-   */
-  private static final String COMPRESSED_SINGLE_ARRAY = "COMPRESSED_SINGLE_ARRAY";
-  /**
    * Double array data store.
    */
   private static final String COMPRESSED_DOUBLE_ARRAY = "COMPRESSED_DOUBLE_ARRAY";
   /**
-   * key type.
-   */
-  private static StoreType keyType;
-  /**
    * value type.
    */
   private static StoreType valueType;
 
   static {
-    String keytype = CarbonProperties.getInstance().getProperty(CarbonCommonConstants.KEYSTORE_TYPE,
-        CarbonCommonConstants.KEYSTORE_TYPE_DEFAULT_VAL);
     String valuetype = CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.VALUESTORE_TYPE,
             CarbonCommonConstants.VALUESTORE_TYPE_DEFAULT_VAL);
 
-    // set key type
-    if (COMPRESSED_SINGLE_ARRAY.equals(keytype)) {
-      keyType = StoreType.COMPRESSED_SINGLE_ARRAY;
-    } else if (SINGLE_ARRAY.equals(keytype)) {
-      keyType = StoreType.SINGLE_ARRAY;
-    } else {
-      keyType = StoreType.COMPRESSED_SINGLE_ARRAY;
-    }
     // set value type
     if (COMPRESSED_DOUBLE_ARRAY.equals(valuetype)) {
       valueType = StoreType.COMPRESSED_DOUBLE_ARRAY;
@@ -89,111 +54,6 @@ public final class StoreFactory {
 
   }
 
-  public static NodeKeyStore createKeyStore(int size, int elementSize, boolean isLeaf,
-      boolean isFileStore, long offset, String fileName, int length, FileHolder fileHolder) {
-    switch (keyType) {
-      case SINGLE_ARRAY:
-
-        if (isFileStore) {
-          return new SingleArrayKeyFileStore(size, elementSize, offset, fileName, length);
-        } else {
-          return new SingleArrayKeyInMemoryStore(size, elementSize, offset, fileName, fileHolder,
-              length);
-        }
-      default:
-
-        if (isLeaf) {
-          if (isFileStore) {
-            return new CompressedSingleArrayKeyFileStore(size, elementSize, offset, fileName,
-                length);
-          } else {
-            return new CompressedSingleArrayKeyInMemoryStore(size, elementSize, offset, fileName,
-                fileHolder, length);
-          }
-        } else {
-          if (isFileStore) {
-            return new SingleArrayKeyFileStore(size, elementSize, offset, fileName, length);
-          } else {
-            return new SingleArrayKeyInMemoryStore(size, elementSize, offset, fileName, fileHolder,
-                length);
-          }
-        }
-    }
-  }
-
-  public static NodeKeyStore createKeyStore(int size, int elementSize, boolean isLeaf) {
-    switch (keyType) {
-      case SINGLE_ARRAY:
-
-        return new SingleArrayKeyInMemoryStore(size, elementSize);
-
-      default:
-
-        if (isLeaf) {
-          return new CompressedSingleArrayKeyInMemoryStore(size, elementSize);
-        } else {
-          return new SingleArrayKeyInMemoryStore(size, elementSize);
-        }
-
-    }
-  }
-
-  public static ColumnarKeyStore createColumnarKeyStore(ColumnarKeyStoreInfo columnarKeyStoreInfo,
-      FileHolder fileHolder, boolean isFileStore) {
-    switch (keyType) {
-      case SINGLE_ARRAY:
-
-        if (isFileStore) {
-          return new UnCompressedColumnarFileKeyStore(columnarKeyStoreInfo);
-        } else {
-          return new UnCompressedColumnarInMemoryStore(columnarKeyStoreInfo, fileHolder);
-        }
-      default:
-
-        if (isFileStore) {
-          return new CompressedColumnarFileKeyStore(columnarKeyStoreInfo);
-        } else {
-          return new CompressedColumnarInMemoryStore(columnarKeyStoreInfo, fileHolder);
-        }
-    }
-  }
-
-  public static NodeMeasureDataStore createDataStore(boolean isFileStore,
-      ValueCompressionModel compressionModel, long[] offset, int[] length, String filePath,
-      FileHolder fileHolder) {
-    switch (valueType) {
-
-      case COMPRESSED_DOUBLE_ARRAY:
-
-        if (isFileStore) {
-          return new DoubleArrayDataFileStore(compressionModel, offset, filePath, length);
-        } else {
-          return new DoubleArrayDataInMemoryStore(compressionModel, offset, length, filePath,
-              fileHolder);
-        }
-
-      case HEAVY_VALUE_COMPRESSION:
-
-        if (isFileStore) {
-          return new HeavyCompressedDoubleArrayDataFileStore(compressionModel, offset, length,
-              filePath);
-        } else {
-          return new HeavyCompressedDoubleArrayDataInMemoryStore(compressionModel, offset, length,
-              filePath, fileHolder);
-        }
-      default:
-
-        if (isFileStore) {
-          return new HeavyCompressedDoubleArrayDataFileStore(compressionModel, offset, length,
-              filePath);
-        } else {
-          return new HeavyCompressedDoubleArrayDataInMemoryStore(compressionModel, offset, length,
-              filePath, fileHolder);
-        }
-
-    }
-  }
-
   public static NodeMeasureDataStore createDataStore(ValueCompressionModel compressionModel) {
     switch (valueType) {
       case COMPRESSED_DOUBLE_ARRAY:
@@ -210,7 +70,6 @@ public final class StoreFactory {
    * enum defined.
    */
   public enum StoreType {
-    SINGLE_ARRAY,
     COMPRESSED_SINGLE_ARRAY,
     COMPRESSED_DOUBLE_ARRAY,
     HEAVY_VALUE_COMPRESSION

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/file/manager/composite/FileData.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/file/manager/composite/FileData.java b/core/src/main/java/org/carbondata/core/file/manager/composite/FileData.java
deleted file mode 100644
index cb4efc6..0000000
--- a/core/src/main/java/org/carbondata/core/file/manager/composite/FileData.java
+++ /dev/null
@@ -1,74 +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.carbondata.core.file.manager.composite;
-
-import org.carbondata.core.writer.HierarchyValueWriterForCSV;
-
-public class FileData extends FileManager {
-
-  /**
-   * Store Path
-   */
-  private String storePath;
-
-  /**
-   * hierarchyValueWriter
-   */
-  private HierarchyValueWriterForCSV hierarchyValueWriter;
-
-  public FileData(String fileName, String storePath) {
-    this.fileName = fileName;
-    this.storePath = storePath;
-  }
-
-  /**
-   * @return Returns the fileName.
-   */
-  public String getFileName() {
-    return fileName;
-  }
-
-  /**
-   * @return Returns the storePath.
-   */
-  public String getStorePath() {
-    return storePath;
-  }
-
-  /**
-   * get Hierarchy Value writer
-   *
-   * @return
-   */
-  public HierarchyValueWriterForCSV getHierarchyValueWriter() {
-    return hierarchyValueWriter;
-  }
-
-  /**
-   * Set Hierarchy Value Writer.
-   *
-   * @param hierarchyValueWriter
-   */
-  public void setHierarchyValueWriter(HierarchyValueWriterForCSV hierarchyValueWriter) {
-    this.hierarchyValueWriter = hierarchyValueWriter;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/file/manager/composite/FileManager.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/file/manager/composite/FileManager.java b/core/src/main/java/org/carbondata/core/file/manager/composite/FileManager.java
deleted file mode 100644
index 7e98bed..0000000
--- a/core/src/main/java/org/carbondata/core/file/manager/composite/FileManager.java
+++ /dev/null
@@ -1,68 +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.carbondata.core.file.manager.composite;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-
-public class FileManager implements IFileManagerComposite {
-  /**
-   * listOfFileData, composite parent which holds the different objects
-   */
-  protected List<IFileManagerComposite> listOfFileData =
-      new ArrayList<IFileManagerComposite>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  protected String fileName;
-
-  @Override public void add(IFileManagerComposite customData) {
-    listOfFileData.add(customData);
-  }
-
-  @Override public void remove(IFileManagerComposite customData) {
-    listOfFileData.remove(customData);
-
-  }
-
-  @Override public IFileManagerComposite get(int i) {
-    return listOfFileData.get(i);
-  }
-
-  /**
-   * Renames the File/Folders
-   */
-  public boolean rename(IFileManagerComposite composite) {
-    return false;
-  }
-
-  @Override public void setName(String name) {
-    this.fileName = name;
-  }
-
-  /**
-   * Return the size
-   */
-  public int size() {
-    return listOfFileData.size();
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/file/manager/composite/IFileManagerComposite.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/file/manager/composite/IFileManagerComposite.java b/core/src/main/java/org/carbondata/core/file/manager/composite/IFileManagerComposite.java
deleted file mode 100644
index a83b9f0..0000000
--- a/core/src/main/java/org/carbondata/core/file/manager/composite/IFileManagerComposite.java
+++ /dev/null
@@ -1,59 +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.carbondata.core.file.manager.composite;
-
-public interface IFileManagerComposite {
-  /**
-   * Add the data which can be either row Folder(Composite) or File
-   *
-   * @param customData
-   */
-  void add(IFileManagerComposite customData);
-
-  /**
-   * Remove the CustomData type object from the IFileManagerComposite object hierarchy.
-   *
-   * @param customData
-   */
-  void remove(IFileManagerComposite customData);
-
-  /**
-   * get the CustomData type object name
-   *
-   * @return CustomDataIntf type
-   */
-  IFileManagerComposite get(int i);
-
-  /**
-   * set the CustomData type object name
-   *
-   * @param name
-   */
-  void setName(String name);
-
-  /**
-   * Get the size
-   *
-   * @return
-   */
-  int size();
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/iterator/CarbonIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/iterator/CarbonIterator.java b/core/src/main/java/org/carbondata/core/iterator/CarbonIterator.java
deleted file mode 100644
index c651a80..0000000
--- a/core/src/main/java/org/carbondata/core/iterator/CarbonIterator.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.carbondata.core.iterator;
-
-import java.util.Iterator;
-
-/**
- * CarbonIterator adds default implement for remove. This is required for Java 7.
- * @param <E>
- */
-public abstract class CarbonIterator<E> implements Iterator<E> {
-
-  @Override public abstract boolean hasNext();
-
-  @Override public abstract E next();
-
-  @Override public void remove() {
-    throw new UnsupportedOperationException("remove");
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/AbstractCarbonLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/AbstractCarbonLock.java b/core/src/main/java/org/carbondata/core/locks/AbstractCarbonLock.java
deleted file mode 100644
index b77cff0..0000000
--- a/core/src/main/java/org/carbondata/core/locks/AbstractCarbonLock.java
+++ /dev/null
@@ -1,77 +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.carbondata.core.locks;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-/**
- * This is the abstract class of the lock implementations.This handles the
- * retrying part of the locking.
- */
-public abstract class AbstractCarbonLock implements ICarbonLock {
-  private int retryCount;
-
-  private int retryTimeout;
-
-  public abstract boolean lock();
-
-  /**
-   * API for enabling the locking of file with retries.
-   */
-  public boolean lockWithRetries() {
-    try {
-      for (int i = 0; i < retryCount; i++) {
-        if (lock()) {
-          return true;
-        } else {
-          Thread.sleep(retryTimeout * 1000L);
-        }
-      }
-    } catch (InterruptedException e) {
-      return false;
-    }
-    return false;
-  }
-
-  /**
-   * Initializes the retry count and retry timeout.
-   * This will determine how many times to retry to acquire lock and the retry timeout.
-   */
-  protected void initRetry() {
-    String retries = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK);
-    try {
-      retryCount = Integer.parseInt(retries);
-    } catch (NumberFormatException e) {
-      retryCount = CarbonCommonConstants.NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK_DEFAULT;
-    }
-
-    String maxTimeout = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK);
-    try {
-      retryTimeout = Integer.parseInt(maxTimeout);
-    } catch (NumberFormatException e) {
-      retryTimeout = CarbonCommonConstants.MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK_DEFAULT;
-    }
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java b/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java
deleted file mode 100644
index 99f4ab1..0000000
--- a/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java
+++ /dev/null
@@ -1,72 +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.carbondata.core.locks;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-/**
- * This class is a Lock factory class which is used to provide lock objects.
- * Using this lock object client can request the lock and unlock.
- */
-public class CarbonLockFactory {
-
-  /**
-   * lockTypeConfigured to check if zookeeper feature is enabled or not for carbon.
-   */
-  private static String lockTypeConfigured;
-
-  static {
-    CarbonLockFactory.updateZooKeeperLockingStatus();
-  }
-
-  /**
-   * This method will determine the lock type.
-   *
-   * @param location
-   * @param lockUsage
-   * @return
-   */
-  public static ICarbonLock getCarbonLockObj(String location, LockUsage lockUsage) {
-    switch (lockTypeConfigured.toUpperCase()) {
-      case CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL:
-        return new LocalFileLock(location, lockUsage);
-
-      case CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER:
-        return new ZooKeeperLocking(location, lockUsage);
-
-      case CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS:
-        return new HdfsFileLock(location, lockUsage);
-
-      default:
-        throw new UnsupportedOperationException("Not supported the lock type");
-    }
-
-  }
-
-  /**
-   * This method will set the zookeeper status whether zookeeper to be used for locking or not.
-   */
-  private static void updateZooKeeperLockingStatus() {
-    lockTypeConfigured = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.LOCK_TYPE, CarbonCommonConstants.LOCK_TYPE_DEFAULT);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/HdfsFileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/HdfsFileLock.java b/core/src/main/java/org/carbondata/core/locks/HdfsFileLock.java
deleted file mode 100644
index 745dfa1..0000000
--- a/core/src/main/java/org/carbondata/core/locks/HdfsFileLock.java
+++ /dev/null
@@ -1,89 +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.carbondata.core.locks;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.IOException;
-
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-
-/**
- * This class is used to handle the HDFS File locking.
- * This is acheived using the concept of acquiring the data out stream using Append option.
- */
-public class HdfsFileLock extends AbstractCarbonLock {
-
-  /**
-   * location hdfs file location
-   */
-  private String location;
-
-  /**
-   * lockUsage is used to determine the type of the lock. according to this the lock
-   * folder will change.
-   */
-  private LockUsage lockUsage;
-
-  private DataOutputStream dataOutputStream;
-
-  /**
-   * @param location
-   * @param lockUsage
-   */
-  public HdfsFileLock(String location, LockUsage lockUsage) {
-    this.location = location;
-    this.lockUsage = lockUsage;
-    this.location = location + File.separator + this.lockUsage;
-    initRetry();
-  }
-
-  /* (non-Javadoc)
-   * @see org.carbondata.core.locks.ICarbonLock#lock()
-   */
-  @Override public boolean lock() {
-    try {
-      if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
-        FileFactory.createNewLockFile(location, FileFactory.getFileType(location));
-      }
-      dataOutputStream =
-          FileFactory.getDataOutputStreamUsingAppend(location, FileFactory.getFileType(location));
-
-      return true;
-
-    } catch (IOException e) {
-      return false;
-    }
-  }
-
-  /* (non-Javadoc)
-   * @see org.carbondata.core.locks.ICarbonLock#unlock()
-   */
-  @Override public boolean unlock() {
-    if (null != dataOutputStream) {
-      try {
-        dataOutputStream.close();
-      } catch (IOException e) {
-        return false;
-      }
-    }
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/ICarbonLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/ICarbonLock.java b/core/src/main/java/org/carbondata/core/locks/ICarbonLock.java
deleted file mode 100644
index af97c2e..0000000
--- a/core/src/main/java/org/carbondata/core/locks/ICarbonLock.java
+++ /dev/null
@@ -1,40 +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.carbondata.core.locks;
-
-/**
- * Carbon Lock Interface which handles the locking and unlocking.
- */
-public interface ICarbonLock {
-
-  /**
-   * Does the unlocking of the acquired lock.
-   *
-   * @return
-   */
-  boolean unlock();
-
-  /**
-   * This will acquire the lock and if it doesnt get then it will retry after the confiured time.
-   *
-   * @return
-   */
-  boolean lockWithRetries();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/LocalFileLock.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/LocalFileLock.java b/core/src/main/java/org/carbondata/core/locks/LocalFileLock.java
deleted file mode 100644
index 362b86f..0000000
--- a/core/src/main/java/org/carbondata/core/locks/LocalFileLock.java
+++ /dev/null
@@ -1,162 +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.carbondata.core.locks;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.nio.channels.FileChannel;
-import java.nio.channels.FileLock;
-import java.nio.channels.OverlappingFileLockException;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-
-/**
- * This class handles the file locking in the local file system.
- * This will be handled using the file channel lock API.
- */
-public class LocalFileLock extends AbstractCarbonLock {
-  /**
-   * location is the location of the lock file.
-   */
-  private String location;
-
-  /**
-   * lockUsage will determine the lock folder. so that similar locks will try to acquire
-   * same lock file.
-   */
-  private LockUsage lockUsage;
-
-  /**
-   * fileOutputStream of the local lock file
-   */
-  private FileOutputStream fileOutputStream;
-
-  /**
-   * channel is the FileChannel of the lock file.
-   */
-  private FileChannel channel;
-
-  /**
-   * fileLock NIO FileLock Object
-   */
-  private FileLock fileLock;
-
-  public static final String tmpPath;
-
-  private String cubeName;
-
-  private String schemaName;
-
-  /**
-   * LOGGER for  logging the messages.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(LocalFileLock.class.getName());
-
-  static {
-    tmpPath = System.getProperty("java.io.tmpdir");
-  }
-
-  /**
-   * @param location
-   * @param lockUsage
-   */
-  public LocalFileLock(String location, LockUsage lockUsage) {
-    this.lockUsage = lockUsage;
-    location = location.replace("\\", "/");
-    String tempStr = location.substring(0, location.lastIndexOf('/'));
-    schemaName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
-
-    cubeName = location.substring(location.lastIndexOf('/') + 1, location.length());
-    this.location =
-        tmpPath + File.separator + schemaName + File.separator + cubeName + File.separator
-            + this.lockUsage;
-    initRetry();
-  }
-
-  /**
-   * Lock API for locking of the file channel of the lock file.
-   *
-   * @return
-   */
-  @Override public boolean lock() {
-    try {
-      String schemaFolderPath = tmpPath + File.separator + schemaName;
-      String cubeFolderPath = schemaFolderPath + File.separator + cubeName;
-      // create dir with schema name in tmp location.
-      if (!FileFactory.isFileExist(schemaFolderPath, FileFactory.getFileType(tmpPath))) {
-        FileFactory.mkdirs(schemaFolderPath, FileFactory.getFileType(tmpPath));
-      }
-
-      // create dir with cube name in tmp location.
-      if (!FileFactory.isFileExist(cubeFolderPath, FileFactory.getFileType(tmpPath))) {
-        FileFactory.mkdirs(cubeFolderPath, FileFactory.getFileType(tmpPath));
-      }
-      if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
-        FileFactory.createNewLockFile(location, FileFactory.getFileType(location));
-      }
-
-      fileOutputStream = new FileOutputStream(location);
-      channel = fileOutputStream.getChannel();
-      try {
-        fileLock = channel.tryLock();
-      } catch (OverlappingFileLockException e) {
-        return false;
-      }
-      if (null != fileLock) {
-        return true;
-      } else {
-        return false;
-      }
-    } catch (IOException e) {
-      return false;
-    }
-
-  }
-
-  /**
-   * Unlock API for unlocking of the acquired lock.
-   *
-   * @return
-   */
-  @Override public boolean unlock() {
-    boolean status;
-    try {
-      if (null != fileLock) {
-        fileLock.release();
-      }
-      status = true;
-    } catch (IOException e) {
-      status = false;
-    } finally {
-      if (null != fileOutputStream) {
-        try {
-          fileOutputStream.close();
-        } catch (IOException e) {
-          LOGGER.error(e.getMessage());
-        }
-      }
-    }
-    return status;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/LockUsage.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/LockUsage.java b/core/src/main/java/org/carbondata/core/locks/LockUsage.java
deleted file mode 100644
index 850b6bf..0000000
--- a/core/src/main/java/org/carbondata/core/locks/LockUsage.java
+++ /dev/null
@@ -1,29 +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.carbondata.core.locks;
-
-/**
- * This enum is used to define the usecase of the lock.
- * Each enum value is one specific lock case.
- */
-public enum LockUsage {
-  METADATA_LOCK,
-  COMPACTION_LOCK;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java b/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java
deleted file mode 100644
index 8b04ac1..0000000
--- a/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java
+++ /dev/null
@@ -1,171 +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.carbondata.core.locks;
-
-import java.util.Collections;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-
-import org.apache.zookeeper.CreateMode;
-import org.apache.zookeeper.KeeperException;
-import org.apache.zookeeper.ZooDefs.Ids;
-import org.apache.zookeeper.ZooKeeper;
-
-/**
- * For Handling the zookeeper locking implementation
- */
-public class ZooKeeperLocking extends AbstractCarbonLock {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ZooKeeperLocking.class.getName());
-
-  /**
-   * zk is the zookeeper client instance
-   */
-  private static ZooKeeper zk;
-
-  /**
-   * zooKeeperLocation is the location in the zoo keeper file system where the locks will be
-   * maintained.
-   */
-  private static final String zooKeeperLocation = CarbonCommonConstants.ZOOKEEPER_LOCATION;
-
-  /**
-   * Unique folder for each cube with SchemaName_CubeName
-   */
-  private final String tableIdFolder;
-
-  /**
-   * lockName is the name of the lock to use. This name should be same for every process that want
-   * to share the same lock
-   */
-  private String lockName;
-
-  /**
-   * lockPath is the unique path created for the each instance of the carbon lock.
-   */
-  private String lockPath;
-
-  private String lockTypeFolder;
-
-  /**
-   * @param lockUsage
-   */
-  public ZooKeeperLocking(String location, LockUsage lockUsage) {
-    this.lockName = CarbonCommonConstants.ZOOKEEPER_LOCK;
-    this.lockTypeFolder = zooKeeperLocation;
-    location = location.replace("\\", "/");
-    String tempStr = location.substring(0, location.lastIndexOf('/'));
-    String schemaName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
-
-    String cubeName = location.substring(location.lastIndexOf('/') + 1, location.length());
-
-    this.tableIdFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + schemaName
-        + '.' + cubeName;
-
-    zk = ZookeeperInit.getInstance().getZookeeper();
-
-    this.lockTypeFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + schemaName
-        + '.' + cubeName + CarbonCommonConstants.FILE_SEPARATOR
-        + lockUsage.toString();
-    try {
-      createBaseNode();
-      // if exists returns null then path doesnt exist. so creating.
-      if (null == zk.exists(this.tableIdFolder, true)) {
-        zk.create(this.tableIdFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-      }
-      // if exists returns null then path doesnt exist. so creating.
-      if (null == zk.exists(this.lockTypeFolder, true)) {
-        zk.create(this.lockTypeFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-      }
-    } catch (KeeperException | InterruptedException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    initRetry();
-  }
-
-  /**
-   * Creating a znode in which all the znodes (lock files )are maintained.
-   */
-  private void createBaseNode() throws KeeperException, InterruptedException {
-    if (null == zk.exists(zooKeeperLocation, true)) {
-      // creating a znode in which all the znodes (lock files )are maintained.
-      zk.create(zooKeeperLocation, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
-    }
-  }
-
-  /**
-   * Handling of the locking mechanism using zoo keeper.
-   */
-  @Override public boolean lock() {
-    try {
-      // create the lock file with lockName.
-      lockPath =
-          zk.create(this.lockTypeFolder + CarbonCommonConstants.FILE_SEPARATOR + lockName, null,
-              Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
-
-      // get the children present in zooKeeperLocation.
-      List<String> nodes = zk.getChildren(this.lockTypeFolder, null);
-
-      // sort the childrens
-      Collections.sort(nodes);
-
-      // here the logic is , for each lock request zookeeper will create a file ending with
-      // incremental digits.
-      // so first request will be 00001 next is 00002 and so on.
-      // if the current request is 00002 and already one previous request(00001) is present then get
-      // children will give both nodes.
-      // after the sort we are checking if the lock path is first or not .if it is first then lock
-      // has been acquired.
-
-      if (lockPath.endsWith(nodes.get(0))) {
-        return true;
-      } else {
-        // if locking failed then deleting the created lock as next time again new lock file will be
-        // created.
-        zk.delete(lockPath, -1);
-        return false;
-      }
-    } catch (KeeperException | InterruptedException e) {
-      LOGGER.error(e, e.getMessage());
-      return false;
-    }
-  }
-
-  /**
-   * @return status where lock file is unlocked or not.
-   */
-  @Override public boolean unlock() {
-    try {
-      // exists will return null if the path doesn't exists.
-      if (null != zk.exists(lockPath, true)) {
-        zk.delete(lockPath, -1);
-        lockPath = null;
-      }
-    } catch (KeeperException | InterruptedException e) {
-      LOGGER.error(e, e.getMessage());
-      return false;
-    }
-    return true;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/locks/ZookeeperInit.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/ZookeeperInit.java b/core/src/main/java/org/carbondata/core/locks/ZookeeperInit.java
deleted file mode 100644
index ba2ad57..0000000
--- a/core/src/main/java/org/carbondata/core/locks/ZookeeperInit.java
+++ /dev/null
@@ -1,64 +0,0 @@
-package org.carbondata.core.locks;
-
-import java.io.IOException;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-
-import org.apache.zookeeper.WatchedEvent;
-import org.apache.zookeeper.Watcher;
-import org.apache.zookeeper.ZooKeeper;
-
-/**
- * This is a singleton class for initialization of zookeeper client.
- */
-public class ZookeeperInit {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ZookeeperInit.class.getName());
-
-  private static ZookeeperInit zooKeeperInit;
-  /**
-   * zk is the zookeeper client instance
-   */
-  private ZooKeeper zk;
-
-  private ZookeeperInit(String zooKeeperUrl) {
-
-    int sessionTimeOut = 100000;
-    try {
-      zk = new ZooKeeper(zooKeeperUrl, sessionTimeOut, new DummyWatcher());
-
-    } catch (IOException e) {
-      LOGGER.error(e.getMessage());
-    }
-
-  }
-
-  public static ZookeeperInit getInstance(String zooKeeperUrl) {
-
-    if (null == zooKeeperInit) {
-      synchronized (ZookeeperInit.class) {
-        if (null == zooKeeperInit) {
-          LOGGER.info("Initiating Zookeeper client.");
-          zooKeeperInit = new ZookeeperInit(zooKeeperUrl);
-        }
-      }
-    }
-    return zooKeeperInit;
-
-  }
-
-  public static ZookeeperInit getInstance() {
-    return zooKeeperInit;
-  }
-
-  public ZooKeeper getZookeeper() {
-    return zk;
-  }
-
-  private static class DummyWatcher implements Watcher {
-    public void process(WatchedEvent event) {
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/metadata/LevelNormalizedProps.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/metadata/LevelNormalizedProps.java b/core/src/main/java/org/carbondata/core/metadata/LevelNormalizedProps.java
deleted file mode 100644
index d1cf630..0000000
--- a/core/src/main/java/org/carbondata/core/metadata/LevelNormalizedProps.java
+++ /dev/null
@@ -1,107 +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.carbondata.core.metadata;
-
-public class LevelNormalizedProps {
-  /**
-   * isLevelNormalized
-   */
-  private boolean isLevelNormalized;
-
-  /**
-   * is this level present in fact file
-   */
-  private boolean isDimInFact;
-
-  /**
-   * hierarchy Name
-   */
-  private String hierName;
-
-  /**
-   * dimension Name
-   */
-  private String dimName;
-
-  /**
-   *
-   */
-  private boolean hasAll;
-
-  /**
-   * Constructor
-   */
-  public LevelNormalizedProps() {
-
-  }
-
-  /**
-   * @return
-   */
-  public String getDimName() {
-    return dimName;
-  }
-
-  /**
-   * @param dimName
-   */
-  public void setDimName(String dimName) {
-    this.dimName = dimName;
-  }
-
-  /**
-   * @return
-   */
-  public String getHierName() {
-    return hierName;
-  }
-
-  /**
-   * @param hierName
-   */
-  public void setHierName(String hierName) {
-    this.hierName = hierName;
-  }
-
-  public boolean isLevelNormalized() {
-    return isLevelNormalized;
-  }
-
-  public void setLevelNormalized(boolean isLevelNormalized) {
-    this.isLevelNormalized = isLevelNormalized;
-  }
-
-  public boolean isDimInFact() {
-    return isDimInFact;
-  }
-
-  public void setDimInFact(boolean isDimInFact) {
-    this.isDimInFact = isDimInFact;
-  }
-
-  public boolean isHasAll() {
-    return hasAll;
-  }
-
-  public void setHasAll(boolean hasAll) {
-    this.hasAll = hasAll;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/metadata/SliceMetaData.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/metadata/SliceMetaData.java b/core/src/main/java/org/carbondata/core/metadata/SliceMetaData.java
deleted file mode 100644
index c0dcfc4..0000000
--- a/core/src/main/java/org/carbondata/core/metadata/SliceMetaData.java
+++ /dev/null
@@ -1,333 +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.carbondata.core.metadata;
-
-import java.io.Serializable;
-import java.util.Arrays;
-import java.util.HashSet;
-
-import org.carbondata.core.keygenerator.KeyGenerator;
-
-public class SliceMetaData implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 3046237866264840878L;
-
-  /**
-   * Array of dimensions declared.
-   */
-  private String[] dimensions;
-
-  private String complexTypeString;
-  /**
-   * actualDimensions
-   */
-  private String[] actualDimensions;
-  /**
-   * Array of measures declared.
-   */
-  private String[] measures;
-  /**
-   * measuresAggregator
-   */
-  private String[] measuresAggregator;
-  /**
-   * Array of newDimensions declared.
-   */
-  private String[] newDimensions;
-  /**
-   * actualNewDimensions
-   */
-  private String[] newActualDimensions;
-  /**
-   * tableNamesToLoadMandatory
-   */
-  private HashSet<String> tableNamesToLoadMandatory;
-  /**
-   * Array of newMeasures declared.
-   */
-  private String[] newMeasures;
-  /**
-   * Array of newMsrDfts declared.
-   */
-  private double[] newMsrDfts;
-  /**
-   * newMeasuresAggregator
-   */
-  private String[] newMeasuresAggregator;
-  /**
-   * heirAnKeySize
-   */
-  private String heirAnKeySize;
-  /**
-   * KeyGenerator declared.
-   */
-  private KeyGenerator keyGenerator;
-  /**
-   * dimLens
-   */
-  private int[] dimLens;
-  /**
-   * actualDimLens
-   */
-  private int[] actualDimLens;
-  /**
-   * newDimLens
-   */
-  private int[] newDimLens;
-  /**
-   * newActualDimLens
-   */
-  private int[] newActualDimLens;
-  /**
-   * oldDimsNewCardinality
-   */
-  private int[] oldDimsNewCardinality;
-  /**
-   * isDimCarinalityChanged
-   */
-  private boolean isDimCarinalityChanged;
-  private String[] newDimsDefVals;
-  private int[] newDimsSurrogateKeys;
-  /**
-   * new keygenerator
-   */
-  private KeyGenerator newKeyGenerator;
-
-  public String getComplexTypeString() {
-    return complexTypeString;
-  }
-
-  public void setComplexTypeString(String complexTypeString) {
-    this.complexTypeString = complexTypeString;
-  }
-
-  public int[] getNewDimLens() {
-    return newDimLens;
-  }
-
-  public void setNewDimLens(int[] newDimLens) {
-    this.newDimLens = newDimLens;
-  }
-
-  public int[] getDimLens() {
-    return dimLens;
-  }
-
-  public void setDimLens(int[] dimLens) {
-    this.dimLens = dimLens;
-  }
-
-  public KeyGenerator getNewKeyGenerator() {
-    return newKeyGenerator;
-  }
-
-  public void setNewKeyGenerator(KeyGenerator newKeyGenerator) {
-    this.newKeyGenerator = newKeyGenerator;
-  }
-
-  public String[] getDimensions() {
-    return dimensions;
-  }
-
-  public void setDimensions(String[] dimensions) {
-    this.dimensions = dimensions;
-  }
-
-  public String[] getMeasures() {
-    return measures;
-  }
-
-  public void setMeasures(String[] measures) {
-    this.measures = measures;
-  }
-
-  public String[] getNewDimensions() {
-    return newDimensions;
-  }
-
-  public void setNewDimensions(String[] newDimensions) {
-    this.newDimensions = newDimensions;
-  }
-
-  public String[] getNewMeasures() {
-    return newMeasures;
-  }
-
-  public void setNewMeasures(String[] newMeasures) {
-    this.newMeasures = newMeasures;
-  }
-
-  public double[] getNewMsrDfts() {
-    return newMsrDfts;
-  }
-
-  public void setNewMsrDfts(double[] newMsrDfts) {
-    this.newMsrDfts = newMsrDfts;
-  }
-
-  public KeyGenerator getKeyGenerator() {
-    return keyGenerator;
-  }
-
-  public void setKeyGenerator(KeyGenerator keyGenerator) {
-    this.keyGenerator = keyGenerator;
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + Arrays.hashCode(dimensions);
-    result = prime * result + ((keyGenerator == null) ? 0 : keyGenerator.hashCode());
-    result = prime * result + Arrays.hashCode(measures);
-    result = prime * result + Arrays.hashCode(newDimensions);
-    result = prime * result + Arrays.hashCode(newMeasures);
-    result = prime * result + Arrays.hashCode(newMsrDfts);
-    result = prime * result + Arrays.hashCode(dimLens);
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj instanceof SliceMetaData) {
-      SliceMetaData other = (SliceMetaData) obj;
-      if (Arrays.equals(dimensions, other.dimensions) && Arrays
-          .equals(measuresAggregator, other.measuresAggregator) && Arrays
-          .equals(dimLens, other.dimLens)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  public boolean isSameAs(SliceMetaData other) {
-    return (Arrays.equals(newDimensions, other.newDimensions) && Arrays
-        .equals(newMeasures, other.newMeasures));
-  }
-
-  public String[] getMeasuresAggregator() {
-    return measuresAggregator;
-  }
-
-  public void setMeasuresAggregator(String[] measuresAggregator) {
-    this.measuresAggregator = measuresAggregator;
-  }
-
-  public String[] getNewMeasuresAggregator() {
-    return newMeasuresAggregator;
-  }
-
-  public void setNewMeasuresAggregator(String[] newMeasuresAggregator) {
-    this.newMeasuresAggregator = newMeasuresAggregator;
-  }
-
-  public String getHeirAnKeySize() {
-    return heirAnKeySize;
-  }
-
-  public void setHeirAnKeySize(String heirAnKeySize) {
-    this.heirAnKeySize = heirAnKeySize;
-  }
-
-  public boolean isDimCarinalityChanged() {
-    return isDimCarinalityChanged;
-  }
-
-  public void setDimCarinalityChanged(boolean isDimCarinalityChanged) {
-    this.isDimCarinalityChanged = isDimCarinalityChanged;
-  }
-
-  public int[] getOldDimsNewCardinality() {
-    return oldDimsNewCardinality;
-  }
-
-  public void setOldDimsNewCardinality(int[] oldDimsNewCardinality) {
-    this.oldDimsNewCardinality = oldDimsNewCardinality;
-  }
-
-  public String[] getNewActualDimensions() {
-    return newActualDimensions;
-  }
-
-  public void setNewActualDimensions(String[] newActualDimensions) {
-    this.newActualDimensions = newActualDimensions;
-  }
-
-  public int[] getNewActualDimLens() {
-    return newActualDimLens;
-  }
-
-  public void setNewActualDimLens(int[] newActualDimLens) {
-    this.newActualDimLens = newActualDimLens;
-  }
-
-  public String[] getActualDimensions() {
-    return actualDimensions;
-  }
-
-  public void setActualDimensions(String[] actualDimensions) {
-    this.actualDimensions = actualDimensions;
-  }
-
-  public int[] getActualDimLens() {
-    return actualDimLens;
-  }
-
-  public void setActualDimLens(int[] actualDimLens) {
-    this.actualDimLens = actualDimLens;
-  }
-
-  public HashSet<String> getTableNamesToLoadMandatory() {
-    return tableNamesToLoadMandatory;
-  }
-
-  public void setTableNamesToLoadMandatory(HashSet<String> tableNamesToLoadMandatory) {
-    this.tableNamesToLoadMandatory = tableNamesToLoadMandatory;
-  }
-
-  /**
-   * return the new dimensions default values
-   */
-  public String[] getNewDimsDefVals() {
-    return newDimsDefVals;
-  }
-
-  /**
-   * set the default values of new dimensions added
-   */
-  public void setNewDimsDefVals(String[] newDimsDefVals) {
-    this.newDimsDefVals = newDimsDefVals;
-  }
-
-  /**
-   * return the surrogate keys of new dimension values
-   */
-  public int[] getNewDimsSurrogateKeys() {
-    return newDimsSurrogateKeys;
-  }
-
-  /**
-   * set the surrogate keys for the new dimension values
-   */
-  public void setNewDimsSurrogateKeys(int[] newDimsSurrogateKeys) {
-    this.newDimsSurrogateKeys = newDimsSurrogateKeys;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/util/CarbonMergerUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonMergerUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonMergerUtil.java
index a359339..e2cdf64 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonMergerUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonMergerUtil.java
@@ -19,19 +19,9 @@
 
 package org.carbondata.core.util;
 
-import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
-
-import org.pentaho.di.core.exception.KettleException;
 
 /**
  * Util class for merge activities of 2 loads.
@@ -44,71 +34,6 @@ public class CarbonMergerUtil {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(CarbonMergerUtil.class.getName());
 
-  public static List<CarbonSliceAndFiles> getSliceAndFilesList(String storeLocation,
-      String tableName, FileType fileType, List<String> loadsToBeMerged) {
-    try {
-      if (!FileFactory.isFileExist(storeLocation, fileType)) {
-        return new ArrayList<CarbonSliceAndFiles>(0);
-      }
-    } catch (IOException e) {
-      LOGGER.error("Error occurred :: " + e.getMessage());
-    }
-    CarbonFile file = FileFactory.getCarbonFile(storeLocation, fileType);
-
-    CarbonFile[] listFiles = CarbonUtil.listFiles(file);
-    if (null == listFiles || listFiles.length < 0) {
-      return new ArrayList<CarbonSliceAndFiles>(0);
-    }
-    Arrays.sort(listFiles, new CarbonFileFolderComparator());
-    listFiles = getMergeFilesList(loadsToBeMerged, listFiles);
-
-    return CarbonUtil.getSliceAndFilesList(tableName, listFiles, fileType);
-  }
-
-  private static CarbonFile[] getMergeFilesList(List<String> loadsToBeMerged,
-      CarbonFile[] listFiles) {
-    CarbonFile[] carbonFile = new CarbonFile[loadsToBeMerged.size()];
-    int i = 0;
-    for (CarbonFile listFile : listFiles) {
-      String loadName = listFile.getName();
-      for (String load : loadsToBeMerged) {
-        if ((CarbonCommonConstants.LOAD_FOLDER + load).equalsIgnoreCase(loadName)) {
-          carbonFile[i++] = listFile;
-        }
-      }
-    }
-    return carbonFile;
-  }
-
-  public static int[] mergeLevelMetadata(String[] sliceLocation, String tableName,
-      String destinationLocation) {
-    int[][] cardinalityOfLoads = new int[sliceLocation.length][];
-    int i = 0;
-    for (String loadFolderLoacation : sliceLocation) {
-      try {
-        cardinalityOfLoads[i++] = CarbonUtil.getCardinalityFromLevelMetadataFile(
-            loadFolderLoacation + '/' + CarbonCommonConstants.LEVEL_METADATA_FILE + tableName
-                + ".metadata");
-      } catch (CarbonUtilException e) {
-        LOGGER
-            .error("Error occurred :: " + e.getMessage());
-      }
-    }
-    int[] MaxCardinality = new int[cardinalityOfLoads[0].length];
-
-    for (int k = 0; k < cardinalityOfLoads[0].length; k++) {
-      MaxCardinality[k] = Math.max(cardinalityOfLoads[0][k], cardinalityOfLoads[1][k]);
-    }
-
-    try {
-      CarbonUtil.writeLevelCardinalityFile(destinationLocation, tableName, MaxCardinality);
-    } catch (KettleException e) {
-      LOGGER.error("Error occurred :: " + e.getMessage());
-    }
-
-    return MaxCardinality;
-  }
-
   public static int[] getCardinalityFromLevelMetadata(String path, String tableName) {
     int[] localCardinality = null;
     try {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/util/CarbonSliceAndFiles.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonSliceAndFiles.java b/core/src/main/java/org/carbondata/core/util/CarbonSliceAndFiles.java
deleted file mode 100644
index 474f247..0000000
--- a/core/src/main/java/org/carbondata/core/util/CarbonSliceAndFiles.java
+++ /dev/null
@@ -1,88 +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.carbondata.core.util;
-
-import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.keygenerator.KeyGenerator;
-
-public class CarbonSliceAndFiles {
-  /**
-   * slice path
-   */
-  private String path;
-
-  /**
-   * slice fact files
-   */
-  private CarbonFile[] sliceFactFilesList;
-
-  private KeyGenerator keyGen;
-
-  /**
-   * This method will be used get the slice fact files
-   *
-   * @return slice fact files
-   */
-  public CarbonFile[] getSliceFactFilesList() {
-    return sliceFactFilesList;
-  }
-
-  /**
-   * This method  will be used to set the slice fact files
-   *
-   * @param sliceFactFilesList
-   */
-  public void setSliceFactFilesList(CarbonFile[] sliceFactFilesList) {
-    this.sliceFactFilesList = sliceFactFilesList;
-  }
-
-  /**
-   * This method will return the slice path
-   *
-   * @return slice path
-   */
-  public String getPath() {
-    return path;
-  }
-
-  /**
-   * This method will be used to set the slice path
-   *
-   * @param path
-   */
-  public void setPath(String path) {
-    this.path = path;
-  }
-
-  /**
-   * @return the keyGen
-   */
-  public KeyGenerator getKeyGen() {
-    return keyGen;
-  }
-
-  /**
-   * @param keyGen the keyGen to set
-   */
-  public void setKeyGen(KeyGenerator keyGen) {
-    this.keyGen = keyGen;
-  }
-
-}


[03/56] [abbrv] incubator-carbondata git commit: [Issue 618]Supported Spark 1.6 in Carbondata (#670)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index b8fb295..8796707 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -25,30 +25,35 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.Job
 import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.agg.{CarbonAverage, CarbonCount}
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, _}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Max, Min, Sum}
+import org.apache.spark.sql.catalyst.util.GenericArrayData
 import org.apache.spark.sql.execution.LeafNode
 import org.apache.spark.sql.hive.CarbonMetastoreCatalog
+import org.apache.spark.sql.types.{DataType, Decimal}
 import org.apache.spark.unsafe.types.UTF8String
 
 import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.carbon.{AbsoluteTableIdentifier}
+import org.carbondata.core.carbon.AbsoluteTableIdentifier
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.util.CarbonProperties
 import org.carbondata.hadoop.CarbonInputFormat
-import org.carbondata.query.aggregator.impl.CountAggregator
+import org.carbondata.query.aggregator.MeasureAggregator
+import org.carbondata.query.aggregator.impl.avg.AbstractAvgAggregator
+import org.carbondata.query.aggregator.impl.count.CountAggregator
+import org.carbondata.query.aggregator.impl.max.{MaxAggregator, MaxBigDecimalAggregator, MaxLongAggregator}
+import org.carbondata.query.aggregator.impl.min.{MinAggregator, MinBigDecimalAggregator, MinLongAggregator}
+import org.carbondata.query.aggregator.impl.sum.{SumBigDecimalAggregator, SumDoubleAggregator, SumLongAggregator}
 import org.carbondata.query.carbon.model.{CarbonQueryPlan, QueryDimension, QueryMeasure, QueryModel, SortOrderType}
 import org.carbondata.query.carbon.result.RowResult
-import org.carbondata.query.expression.{ColumnExpression => CarbonColumnExpression}
-import org.carbondata.query.expression.{Expression => CarbonExpression}
-import org.carbondata.query.expression.{LiteralExpression => CarbonLiteralExpression}
+import org.carbondata.query.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
 import org.carbondata.query.expression.arithmetic.{AddExpression, DivideExpression, MultiplyExpression, SubstractExpression}
 import org.carbondata.query.expression.conditional._
 import org.carbondata.query.expression.logical.{AndExpression, OrExpression}
 import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
-import org.carbondata.spark.agg._
-import org.carbondata.spark.KeyVal
-import org.carbondata.spark.KeyValImpl
+import org.carbondata.spark.{KeyVal, KeyValImpl}
 import org.carbondata.spark.rdd.CarbonQueryRDD
 import org.carbondata.spark.util.{CarbonScalaUtil, QueryPlanUtil}
 
@@ -70,14 +75,35 @@ case class CarbonTableScan(
   var outputColumns = scala.collection.mutable.MutableList[Attribute]()
   var extraPreds: Seq[Expression] = Nil
   val allDims = new scala.collection.mutable.HashSet[String]()
-  // val carbonTable = CarbonMetadata.getInstance().getCarbonTable(cubeName)
   @transient val carbonCatalog = sqlContext.catalog.asInstanceOf[CarbonMetastoreCatalog]
 
-  def processAggregateExpr(plan: CarbonQueryPlan, currentAggregate: AggregateExpression1,
-      queryOrder: Int): Int = {
-
+  def processAggregateExpr(plan: CarbonQueryPlan,
+      currentAggregate: AggregateExpression,
+      queryOrder: Int,
+      aggCount: Int): Int = {
     currentAggregate match {
-      case SumCarbon(posLiteral@PositionLiteral(attr: AttributeReference, _), _) =>
+      case AggregateExpression(Sum(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
+        outputColumns += attr
+        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
+        if (msrs.nonEmpty) {
+          val m1 = new QueryMeasure(attr.name)
+          m1.setAggregateFunction(CarbonCommonConstants.SUM)
+          m1.setQueryOrder(queryOrder)
+          plan.addMeasure(m1)
+        } else {
+          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
+          if (dims.nonEmpty) {
+            val d1 = new QueryDimension(attr.name)
+            d1.setQueryOrder(queryOrder)
+            plan.addAggDimAggInfo(d1.getColumnName, "sum", d1.getQueryOrder)
+          }
+        }
+        p.setPosition(queryOrder + aggCount)
+        queryOrder + 1
+
+      case AggregateExpression(
+        Sum(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
+        outputColumns += attr
         val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
         if (msrs.nonEmpty) {
           val m1 = new QueryMeasure(attr.name)
@@ -92,10 +118,12 @@ case class CarbonTableScan(
             plan.addAggDimAggInfo(d1.getColumnName, "sum", d1.getQueryOrder)
           }
         }
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
-      case CountCarbon(posLiteral@PositionLiteral(attr: AttributeReference, _)) =>
+      case AggregateExpression(
+        CarbonCount(p@PositionLiteral(attr: AttributeReference, _), None), _, false) =>
+        outputColumns += attr
         val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
         if (msrs.nonEmpty) {
           val m1 = new QueryMeasure(attr.name)
@@ -110,33 +138,28 @@ case class CarbonTableScan(
             plan.addAggDimAggInfo(d1.getColumnName, "count", d1.getQueryOrder)
           }
         }
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
-      case CountCarbon(posLiteral@PositionLiteral(Literal(star, _), _)) if star == "*" =>
+      case AggregateExpression(
+        CarbonCount(lt: Literal, Some(p@PositionLiteral(attr: AttributeReference, _))), _, false)
+        if lt.value == "*" || lt.value == 1 =>
+        outputColumns += attr
         val m1 = new QueryMeasure("count(*)")
         m1.setAggregateFunction(CarbonCommonConstants.COUNT)
         m1.setQueryOrder(queryOrder)
-        posLiteral.setPosition(queryOrder)
         plan.addMeasure(m1)
         plan.setCountStartQuery(true)
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
-      case curr@CountCarbon(posLiteral@PositionLiteral(one, _)) =>
-        val m1 = new QueryMeasure("count(*)")
-        m1.setAggregateFunction(CarbonCommonConstants.COUNT)
-        m1.setQueryOrder(queryOrder)
-        posLiteral.setPosition(queryOrder)
-        plan.addMeasure(m1)
-        plan.setCountStartQuery(true)
-        posLiteral.setPosition(queryOrder)
-        queryOrder + 1
-      case CountDistinctCarbon(posLiteral@PositionLiteral(attr: AttributeReference, _)) =>
+      case AggregateExpression(
+        CarbonAverage(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
+        outputColumns += attr
         val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
         if (msrs.nonEmpty) {
           val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.DISTINCT_COUNT)
+          m1.setAggregateFunction(CarbonCommonConstants.AVERAGE)
           m1.setQueryOrder(queryOrder)
           plan.addMeasure(m1)
         } else {
@@ -144,13 +167,15 @@ case class CarbonTableScan(
           if (dims.nonEmpty) {
             val d1 = new QueryDimension(attr.name)
             d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "distinct-count", d1.getQueryOrder)
+            plan.addAggDimAggInfo(d1.getColumnName, "avg", d1.getQueryOrder)
           }
         }
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
-      case AverageCarbon(posLiteral@PositionLiteral(attr: AttributeReference, _), _) =>
+      case AggregateExpression(
+        CarbonAverage(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
+        outputColumns += attr
         val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
         if (msrs.nonEmpty) {
           val m1 = new QueryMeasure(attr.name)
@@ -165,10 +190,31 @@ case class CarbonTableScan(
             plan.addAggDimAggInfo(d1.getColumnName, "avg", d1.getQueryOrder)
           }
         }
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
+        queryOrder + 1
+
+      case AggregateExpression(Min(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
+        outputColumns += attr
+        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
+        if (msrs.nonEmpty) {
+          val m1 = new QueryMeasure(attr.name)
+          m1.setAggregateFunction(CarbonCommonConstants.MIN)
+          m1.setQueryOrder(queryOrder)
+          plan.addMeasure(m1)
+        } else {
+          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
+          if (dims != null) {
+            val d1 = new QueryDimension(attr.name)
+            d1.setQueryOrder(queryOrder)
+            plan.addAggDimAggInfo(d1.getColumnName, "min", d1.getQueryOrder)
+          }
+        }
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
-      case MinCarbon(posLiteral@PositionLiteral(attr: AttributeReference, _), _) =>
+      case AggregateExpression(
+        Min(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
+        outputColumns += attr
         val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
         if (msrs.nonEmpty) {
           val m1 = new QueryMeasure(attr.name)
@@ -183,10 +229,11 @@ case class CarbonTableScan(
             plan.addAggDimAggInfo(d1.getColumnName, "min", d1.getQueryOrder)
           }
         }
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
-      case MaxCarbon(posLiteral@PositionLiteral(attr: AttributeReference, _), _) =>
+      case AggregateExpression(Max(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
+        outputColumns += attr
         val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
         if (msrs.nonEmpty) {
           val m1 = new QueryMeasure(attr.name)
@@ -201,26 +248,27 @@ case class CarbonTableScan(
             plan.addAggDimAggInfo(d1.getColumnName, "max", d1.getQueryOrder)
           }
         }
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
-      case SumDistinctCarbon(posLiteral@PositionLiteral(attr: AttributeReference, _), _) =>
+      case AggregateExpression(
+        Max(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
+        outputColumns += attr
         val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
         if (msrs.nonEmpty) {
           val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.SUM_DISTINCT)
+          m1.setAggregateFunction(CarbonCommonConstants.MAX)
           m1.setQueryOrder(queryOrder)
           plan.addMeasure(m1)
         } else {
           val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims != null) {
-            //            plan.removeDimensionFromDimList(dims(0));
+          if (dims.nonEmpty) {
             val d1 = new QueryDimension(attr.name)
             d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "sum-distinct", queryOrder)
+            plan.addAggDimAggInfo(d1.getColumnName, "max", d1.getQueryOrder)
           }
         }
-        posLiteral.setPosition(queryOrder)
+        p.setPosition(queryOrder + aggCount)
         queryOrder + 1
 
       case _ => throw new
@@ -239,7 +287,6 @@ case class CarbonTableScan(
         val carbonDimension = carbonTable.getDimensionByName(carbonTable.getFactTableName
           , attr.name)
         if (carbonDimension != null) {
-          // TODO if we can add ordina in carbonDimension, it will be good
           allDims += attr.name
           val dim = new QueryDimension(attr.name)
           dim.setQueryOrder(queryOrder)
@@ -257,50 +304,63 @@ case class CarbonTableScan(
         }
       })
     queryOrder = 0
+
+    // It is required to calculate as spark aggregators uses joined row with the current aggregates.
+    val aggCount = aggExprs match {
+      case Some(a: Seq[Expression]) =>
+        a.map {
+          case Alias(AggregateExpression(CarbonAverage(_), _, _), name) => 2
+          case Alias(agg: AggregateExpression, name) => 1
+          case _ => 0
+        }.reduceLeftOption((left, right) => left + right).getOrElse(0)
+      case _ => 0
+    }
     // Separately handle group by columns, known or unknown partial aggregations and other
     // expressions. All single column & known aggregate expressions will use native aggregates for
     // measure and dimensions
     // Unknown aggregates & Expressions will use custom aggregator
+
     aggExprs match {
       case Some(a: Seq[Expression]) if !forceDetailedQuery =>
         a.foreach {
           case attr@AttributeReference(_, _, _, _) => // Add all the references to carbon query
-            val carbonDimension = selectedDims
-              .filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-            if (carbonDimension.nonEmpty) {
-              val dim = new QueryDimension(attr.name)
-              dim.setQueryOrder(queryOrder)
-              plan.addDimension(dim)
-              queryOrder = queryOrder + 1
-            } else {
-              val carbonMeasure = selectedMsrs
-                .filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-              if (carbonMeasure.nonEmpty) {
-                // added by vishal as we are adding for dimension so need to add to measure list
-                // Carbon does not support group by on measure column so throwing exception to
-                // make it detail query
-                throw new
-                    Exception("Some Aggregate functions cannot be pushed, force to detailequery")
-              }
-              else {
-                // Some unknown attribute name is found. this may be a derived column.
-                // So, let's fall back to detailed query flow
-                throw new Exception(
-                  "Some attributes referred looks derived columns. So, force to detailequery " +
-                  attr.name)
-              }
-            }
+            addCarbonColumn(attr)
             outputColumns += attr
-          case par: Alias if par.children.head.isInstanceOf[AggregateExpression1] =>
-            outputColumns += par.toAttribute
-            queryOrder = processAggregateExpr(plan,
-              par.children.head.asInstanceOf[AggregateExpression1], queryOrder)
-
+          case al@ Alias(agg: AggregateExpression, name) =>
+            queryOrder = processAggregateExpr(plan, agg, queryOrder, aggCount)
           case _ => forceDetailedQuery = true
         }
       case _ => forceDetailedQuery = true
     }
 
+    def addCarbonColumn(attr: Attribute): Unit = {
+      val carbonDimension = selectedDims
+        .filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
+      if (carbonDimension.nonEmpty) {
+        val dim = new QueryDimension(attr.name)
+        dim.setQueryOrder(queryOrder)
+        plan.addDimension(dim)
+        queryOrder = queryOrder + 1
+      } else {
+        val carbonMeasure = selectedMsrs
+          .filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
+        if (carbonMeasure.nonEmpty) {
+          // added by vishal as we are adding for dimension so need to add to measure list
+          // Carbon does not support group by on measure column so throwing exception to
+          // make it detail query
+          throw new
+              Exception("Some Aggregate functions cannot be pushed, force to detailequery")
+        }
+        else {
+          // Some unknown attribute name is found. this may be a derived column.
+          // So, let's fall back to detailed query flow
+          throw new Exception(
+            "Some attributes referred looks derived columns. So, force to detailequery " +
+            attr.name)
+        }
+      }
+    }
+
     if (forceDetailedQuery) {
       // First clear the model if Msrs, Expressions and AggDimAggInfo filled
       plan.getDimensions.clear()
@@ -313,6 +373,12 @@ case class CarbonTableScan(
       selectedMsrs.foreach(plan.addMeasure)
     }
     else {
+      attributes.foreach { attr =>
+        if (!outputColumns.exists(_.name.equals(attr.name))) {
+          addCarbonColumn(attr)
+          outputColumns += attr
+        }
+      }
       attributes = outputColumns
     }
 
@@ -322,16 +388,13 @@ case class CarbonTableScan(
     sortExprs match {
       case Some(a: Seq[SortOrder]) =>
         a.foreach {
-          case SortOrder(SumCarbon(attr: AttributeReference, _), order) => plan.getMeasures
-            .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name)).head
-            .setSortOrder(getSortDirection(order))
-          case SortOrder(CountCarbon(attr: AttributeReference), order) => plan.getMeasures
+          case SortOrder(Sum(attr: AttributeReference), order) => plan.getMeasures
             .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name)).head
             .setSortOrder(getSortDirection(order))
-          case SortOrder(CountDistinctCarbon(attr: AttributeReference), order) => plan.getMeasures
+          case SortOrder(CarbonCount(attr: AttributeReference, _), order) => plan.getMeasures
             .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name)).head
             .setSortOrder(getSortDirection(order))
-          case SortOrder(AverageCarbon(attr: AttributeReference, _), order) => plan.getMeasures
+          case SortOrder(CarbonAverage(attr: AttributeReference), order) => plan.getMeasures
             .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name)).head
             .setSortOrder(getSortDirection(order))
           case SortOrder(attr: AttributeReference, order) =>
@@ -476,7 +539,7 @@ case class CarbonTableScan(
   def inputRdd: CarbonQueryRDD[CarbonKey, CarbonValue] = {
     val LOG = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
     // Update the FilterExpressions with extra conditions added through join pushdown
-    if (extraPreds.nonEmpty) {
+    if (extraPreds.nonEmpty) {attributes
       val exps = preProcessExpressions(extraPreds)
       val expressionVal = transformExpression(exps.head)
       val oldExpressionVal = buildCarbonPlan.getFilterExpression
@@ -514,13 +577,35 @@ case class CarbonTableScan(
     big
   }
 
+
+  override def outputsUnsafeRows: Boolean = false
+
   def doExecute(): RDD[InternalRow] = {
     def toType(obj: Any): Any = {
       obj match {
         case s: String => UTF8String.fromString(s)
+        case avg: AbstractAvgAggregator =>
+          if (avg.isFirstTime) {
+            null
+          } else {
+            new GenericArrayData(avg.getAvgState.asInstanceOf[Array[Any]])
+          }
+        case c: CountAggregator => c.getLongValue
+        case s: SumDoubleAggregator => s.getDoubleValue
+        case s: SumBigDecimalAggregator => Decimal(s.getBigDecimalValue)
+        case s: SumLongAggregator => s.getLongValue
+        case m: MaxBigDecimalAggregator => Decimal(m.getBigDecimalValue)
+        case m: MaxLongAggregator => m.getLongValue
+        case m: MaxAggregator => toType(m.getValueObject)
+        case m: MinBigDecimalAggregator => Decimal(m.getBigDecimalValue)
+        case m: MinLongAggregator => m.getLongValue
+        case m: MinAggregator => toType(m.getValueObject)
+        case m: MeasureAggregator => toType(m.getValueObject)
         case _ => obj
       }
     }
+
+//    val unsafeProjection = UnsafeProjection.create(attributes.map(_.dataType).toArray)
     // count(*) query executed in driver by querying from Btree
     if (isCountQuery) {
       val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
@@ -531,20 +616,26 @@ case class CarbonTableScan(
       val countAgg = new CountAggregator()
       countAgg.setNewValue(rowCount)
       sparkContext.parallelize(
-        Seq(new GenericMutableRow(Seq(countAgg).toArray.asInstanceOf[Array[Any]]))
+        Seq(new GenericMutableRow(Seq(countAgg.getLongValue).toArray.asInstanceOf[Array[Any]]))
       )
     } else {
       // all the other queries are sent to executor
-      inputRdd.map { row =>
-        val dims = row._1.getKey.map(toType)
-        val values = dims
-        new GenericMutableRow(values.asInstanceOf[Array[Any]])
+      inputRdd.mapPartitions { iter =>
+        new Iterator[InternalRow] {
+//          val unsafeProjection = UnsafeProjection.create(attributes.map(_.dataType).toArray)
+          override def hasNext: Boolean = iter.hasNext
+
+          override def next(): InternalRow = {
+            new GenericMutableRow(iter.next()._1.getKey.map(toType))
+          }
+        }
       }
     }
   }
 
   /**
-   * return true if query is count query
+   * return true if query is count queryUtils
+ *
    * @return
    */
   def isCountQuery: Boolean = {
@@ -560,6 +651,5 @@ case class CarbonTableScan(
   def output: Seq[Attribute] = {
     attributes
   }
-
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawAggregate.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawAggregate.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawAggregate.scala
deleted file mode 100644
index 27df43f..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawAggregate.scala
+++ /dev/null
@@ -1,239 +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
-
-import java.util.HashMap
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.errors._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.execution.{SparkPlan, UnaryNode}
-import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.unsafe.types.UTF8String
-
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper
-
-/**
- * :: DeveloperApi ::
- * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each
- * group.
- *
- * @param partial              if true then aggregation is done partially on local data without
- *                             shuffling to
- *                             ensure all values where `groupingExpressions` are equal are present.
- * @param groupingExpressions  expressions that are evaluated to determine grouping.
- * @param aggregateExpressions expressions that are computed for each group.
- * @param child                the input data source.
- */
-@DeveloperApi
-case class CarbonRawAggregate(
-  partial: Boolean,
-  groupingExpressions: Seq[Expression],
-  aggregateExpressions: Seq[NamedExpression],
-  child: SparkPlan)
-  extends UnaryNode {
-
-  override private[sql] lazy val metrics = Map(
-    "numInputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of input rows"),
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  override def requiredChildDistribution: List[Distribution] = {
-    if (partial) {
-      UnspecifiedDistribution :: Nil
-    } else {
-      if (groupingExpressions == Nil) {
-        AllTuples :: Nil
-      } else {
-        ClusteredDistribution(groupingExpressions) :: Nil
-      }
-    }
-  }
-
-  override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute)
-
-  /**
-   * An aggregate that needs to be computed for each row in a group.
-   *
-   * @param unbound         Unbound version of this aggregate, used for result substitution.
-   * @param aggregate       A bound copy of this aggregate used to create a new aggregation buffer.
-   * @param resultAttribute An attribute used to refer to the result of this aggregate in the final
-   *                        output.
-   */
-  case class ComputedAggregate(
-    unbound: AggregateExpression1,
-    aggregate: AggregateExpression1,
-    resultAttribute: AttributeReference)
-
-  /** A list of aggregates that need to be computed for each group. */
-  private[this] val computedAggregates = aggregateExpressions.flatMap { agg =>
-    agg.collect {
-      case a: AggregateExpression1 =>
-        ComputedAggregate(
-          a,
-          BindReferences.bindReference(a, child.output),
-          AttributeReference(s"aggResult:$a", a.dataType, a.nullable)())
-    }
-  }.toArray
-
-  val order = aggregateExpressions.flatMap { agg =>
-    agg.collect {
-      case a: Expression =>
-        BindReferences.bindReference(a, child.output).collect {
-          case b: BoundReference => b.ordinal
-        }
-    }.flatMap(f => f)
-  }.toArray
-
-  /** The schema of the result of all aggregate evaluations */
-  private[this] val computedSchema = computedAggregates.map(_.resultAttribute)
-
-  /** Creates a new aggregate buffer for a group. */
-  private[this] def newAggregateBuffer(): Array[AggregateFunction1] = {
-    val buffer = new Array[AggregateFunction1](computedAggregates.length)
-    var i = 0
-    while (i < computedAggregates.length) {
-      buffer(i) = computedAggregates(i).aggregate.newInstance()
-      i += 1
-    }
-    buffer
-  }
-
-  /** Named attributes used to substitute grouping attributes into the final result. */
-  private[this] val namedGroups = groupingExpressions.map {
-    case ne: NamedExpression => ne -> ne.toAttribute
-    case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute
-  }
-
-  /**
-   * A map of substitutions that are used to insert the aggregate expressions and grouping
-   * expression into the final result expression.
-   */
-  private[this] val resultMap =
-    (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute } ++ namedGroups).toMap
-
-  /**
-   * Substituted version of aggregateExpressions expressions which are used to compute final
-   * output rows given a group and the result of all aggregate computations.
-   */
-  private[this] val resultExpressions = aggregateExpressions.map { agg =>
-    agg.transform {
-      case e: Expression if resultMap.contains(e) => resultMap(e)
-    }
-  }
-
-  protected override def doExecute(): RDD[InternalRow] = {
-    attachTree(this, "execute") {
-      val numInputRows = longMetric("numInputRows")
-      val numOutputRows = longMetric("numOutputRows")
-      val pOrder = order.zipWithIndex.map {f =>
-        if (f._2 > 0 && order(f._2-1) == f._1) {
-          f._1 + 1
-        } else {
-          f._1
-        }
-      }
-      if (groupingExpressions.isEmpty) {
-        child.execute().mapPartitions { iter =>
-          val buffer = newAggregateBuffer()
-          var currentRow: CarbonRawMutableRow = null
-          while (iter.hasNext) {
-            currentRow = iter.next().asInstanceOf[CarbonRawMutableRow]
-            while (currentRow.hasNext) {
-              numInputRows += 1
-              var i = 0
-              while (i < buffer.length) {
-                buffer(i).update(currentRow)
-                i += 1
-              }
-              currentRow.next()
-            }
-          }
-          val resultProjection = new InterpretedProjection(resultExpressions, computedSchema)
-          val aggregateResults = new GenericMutableRow(computedAggregates.length)
-
-          var i = 0
-          while (i < buffer.length) {
-            aggregateResults(i) = buffer(i).eval(EmptyRow)
-            i += 1
-          }
-
-          numOutputRows += 1
-          Iterator(resultProjection(aggregateResults))
-        }
-      } else {
-        child.execute().mapPartitions { iter =>
-          val hashTable = new HashMap[ByteArrayWrapper, Array[AggregateFunction1]](10000)
-          var currentRow: CarbonRawMutableRow = null
-          while (iter.hasNext) {
-            currentRow = iter.next().asInstanceOf[CarbonRawMutableRow]
-            while (currentRow.hasNext) {
-              numInputRows += 1
-              val currentGroup = currentRow.getKey
-              var currentBuffer = hashTable.get(currentGroup)
-              if (currentBuffer == null) {
-                currentBuffer = newAggregateBuffer()
-                hashTable.put(currentGroup, currentBuffer)
-              }
-
-              var i = 0
-              while (i < currentBuffer.length) {
-                currentBuffer(i).update(currentRow)
-                i += 1
-              }
-              currentRow.next()
-            }
-          }
-
-          new Iterator[InternalRow] {
-            private[this] val hashTableIter = hashTable.entrySet().iterator()
-            private[this] val aggregateResults = new Array[Any](computedAggregates.length)
-
-            override final def hasNext: Boolean = hashTableIter.hasNext
-
-            override final def next(): InternalRow = {
-              val currentEntry = hashTableIter.next()
-              val currentGroup = currentEntry.getKey
-              val currentBuffer = currentEntry.getValue
-              numOutputRows += 1
-              var i = 0
-              while (i < currentBuffer.length) {
-                // Evaluating an aggregate buffer returns the result.  No row is required since we
-                // already added all rows in the group using update.
-                aggregateResults(i) = currentBuffer(i).eval(EmptyRow)
-                i += 1
-              }
-              new GenericMutableRow(
-                currentRow.parseKey(
-                  currentGroup, aggregateResults.asInstanceOf[Array[Object]], pOrder).map(toType))
-            }
-          }
-        }
-      }
-    }
-  }
-
-  def toType(obj: Any): Any = {
-    obj match {
-      case s: String => UTF8String.fromString(s)
-      case _ => obj
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala
index 8912def..2005300 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala
@@ -17,7 +17,6 @@
 
 package org.apache.spark.sql
 
-import java.util
 import java.util.ArrayList
 
 import scala.collection.JavaConverters._
@@ -27,18 +26,19 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.LeafNode
 import org.apache.spark.sql.hive.CarbonMetastoreCatalog
 import org.apache.spark.sql.types.{DataType, Decimal}
 import org.apache.spark.unsafe.types.UTF8String
 
-import org.carbondata.core.carbon.{AbsoluteTableIdentifier}
+import org.carbondata.core.carbon.AbsoluteTableIdentifier
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.util.CarbonProperties
 import org.carbondata.query.carbon.model._
 import org.carbondata.query.carbon.result.BatchRawResult
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper
-import org.carbondata.spark.{CarbonFilters, RawKeyVal, RawKeyValImpl}
+import org.carbondata.spark.{CarbonFilters, RawKey, RawKeyImpl, RawKeyVal, RawKeyValImpl}
 import org.carbondata.spark.rdd.CarbonRawQueryRDD
 
 
@@ -54,7 +54,7 @@ case class CarbonRawTableScan(
   val selectedMsrs = scala.collection.mutable.MutableList[QueryMeasure]()
   @transient val carbonCatalog = ocRaw.catalog.asInstanceOf[CarbonMetastoreCatalog]
 
-  val attributesNeedToDecode = new util.HashSet[AttributeReference]()
+  val attributesNeedToDecode = new java.util.HashSet[AttributeReference]()
   val unprocessedExprs = new ArrayBuffer[Expression]()
 
   val buildCarbonPlan: CarbonQueryPlan = {
@@ -64,7 +64,7 @@ case class CarbonRawTableScan(
     val measures = carbonTable.getMeasureByTableName(carbonTable.getFactTableName)
     val dimAttr = new Array[Attribute](dimensions.size())
     val msrAttr = new Array[Attribute](measures.size())
-    attributesRaw.map { attr =>
+    attributesRaw.foreach { attr =>
       val carbonDimension =
         carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
       if(carbonDimension != null) {
@@ -78,10 +78,10 @@ case class CarbonRawTableScan(
       }
     }
 
-    attributesRaw = (dimAttr.filter(f => f != null)) ++ (msrAttr.filter(f => f != null))
+    attributesRaw = dimAttr.filter(f => f != null) ++ msrAttr.filter(f => f != null)
 
     var queryOrder: Integer = 0
-    attributesRaw.map { attr =>
+    attributesRaw.foreach { attr =>
         val carbonDimension =
           carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
         if (carbonDimension != null) {
@@ -104,7 +104,7 @@ case class CarbonRawTableScan(
     aggExprsRaw match {
       case Some(aggExprs) =>
         aggExprs.foreach {
-          case Alias(agg: AggregateExpression1, name) =>
+          case Alias(agg: AggregateExpression, name) =>
             agg.collect {
               case attr: AttributeReference =>
                 val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
@@ -136,8 +136,11 @@ case class CarbonRawTableScan(
 
   def processFilterExpressions(plan: CarbonQueryPlan) {
     if (dimensionPredicatesRaw.nonEmpty) {
-      val expressionVal = CarbonFilters
-        .processExpression(dimensionPredicatesRaw, attributesNeedToDecode, unprocessedExprs)
+      val expressionVal = CarbonFilters.processExpression(
+        dimensionPredicatesRaw,
+        attributesNeedToDecode,
+        unprocessedExprs,
+        carbonTable)
       expressionVal match {
         case Some(ce) =>
           // adding dimension used in expression in querystats
@@ -181,14 +184,14 @@ case class CarbonRawTableScan(
   }
 
 
-  def inputRdd: CarbonRawQueryRDD[BatchRawResult, Any] = {
+  def inputRdd: CarbonRawQueryRDD[Array[Any], Any] = {
 
     val conf = new Configuration()
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
     buildCarbonPlan.getDimAggregatorInfos.clear()
     val model = QueryModel.createModel(
       absoluteTableIdentifier, buildCarbonPlan, carbonTable)
-    val kv: RawKeyVal[BatchRawResult, Any] = new RawKeyValImpl()
+    val kv: RawKey[Array[Any], Any] = new RawKeyImpl()
     // setting queryid
     buildCarbonPlan.setQueryId(ocRaw.getConf("queryId", System.nanoTime() + ""))
 
@@ -208,6 +211,9 @@ case class CarbonRawTableScan(
     big
   }
 
+
+  override def outputsUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
+
   override def doExecute(): RDD[InternalRow] = {
     def toType(obj: Any): Any = {
       obj match {
@@ -215,19 +221,18 @@ case class CarbonRawTableScan(
         case _ => obj
       }
     }
-
-    if (useBinaryAggregator) {
-      inputRdd.map { row =>
-        //      val dims = row._1.map(toType)
-        new CarbonRawMutableRow(row._1.getAllRows, row._1.getQuerySchemaInfo)
-      }
-    } else {
-      inputRdd.flatMap { row =>
-        val buffer = new ArrayBuffer[GenericMutableRow]()
-        while (row._1.hasNext) {
-          buffer += new GenericMutableRow(row._1.next().map(toType))
-        }
-        buffer
+    val outUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
+    inputRdd.mapPartitions { iter =>
+      val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
+      new Iterator[InternalRow] {
+        override def hasNext: Boolean = iter.hasNext
+
+        override def next(): InternalRow =
+          if (outUnsafeRows) {
+            unsafeProjection(new GenericMutableRow(iter.next()._1.map(toType)))
+          } else {
+            new GenericMutableRow(iter.next()._1.map(toType))
+          }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index ecaa711..a72aaa1 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -1231,12 +1231,12 @@ class CarbonSqlParser()
       case ef ~ db ~ tbl =>
         val tblIdentifier = db match {
           case Some(dbName) =>
-            Seq(dbName, tbl.toLowerCase())
+            TableIdentifier(tbl.toLowerCase, Some(dbName))
           case None =>
-            Seq(tbl.toLowerCase())
+            TableIdentifier(tbl.toLowerCase)
         }
         if (ef.isDefined && "FORMATTED".equalsIgnoreCase(ef.get)) {
-          new DescribeFormattedCommand("describe formatted " + tblIdentifier.mkString("."),
+          new DescribeFormattedCommand("describe formatted " + tblIdentifier,
             tblIdentifier)
         }
         else {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownCarbonAggregator.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownCarbonAggregator.scala b/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownCarbonAggregator.scala
deleted file mode 100644
index d5d8bee..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownCarbonAggregator.scala
+++ /dev/null
@@ -1,179 +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
-
-import java.io.{DataInput, DataOutput}
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql.catalyst.expressions.{AggregateExpression1, AggregateFunction1, GenericMutableRow}
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk
-import org.carbondata.core.datastorage.store.dataholder.CarbonReadDataHolder
-import org.carbondata.query.aggregator.{CustomMeasureAggregator, MeasureAggregator}
-import org.carbondata.query.carbonfilterinterface.RowIntf
-import org.carbondata.query.expression.ColumnExpression
-
-/**
- * Custom Aggregator serialized and used to pushdown all aggregate functions from spark layer with
- * expressions to Carbon layer
- */
-@SerialVersionUID(-3787749110799088697L)
-class SparkUnknownCarbonAggregator(partialAggregate: AggregateExpression1)
-  extends CustomMeasureAggregator {
-
-  val result = scala.collection.mutable.MutableList[GenericMutableRow]()
-  @transient var partialFunction: AggregateFunction1 = null
-
-  @transient var allColumns: java.util.List[ColumnExpression] = null
-  var isRowsAggregated: Boolean = false
-
-  def this() = this(null) // For serializattion
-
-  override def agg(newVal: Double): Unit = {
-
-    throw new UnsupportedOperationException("agg(double) is not a valid method for aggregation")
-  }
-
-  override def agg(newVal: Any): Unit = {
-    throw new UnsupportedOperationException("agg(Object) is not a valid method for aggregation")
-  }
-
-  override def agg(newVal: MeasureColumnDataChunk, index: Int): Unit = {
-    throw new UnsupportedOperationException(
-      "agg(CarbonReadDataHolder, int) is not a valid method for aggregation"
-    )
-  }
-
-  override def getByteArray: Array[Byte] = {
-    throw new UnsupportedOperationException("getByteArray  is not implemented yet")
-  }
-
-  override def getDoubleValue: java.lang.Double = {
-    throw new UnsupportedOperationException("getValue() is not a valid method for result")
-  }
-
-  override def getLongValue: java.lang.Long = {
-    throw new UnsupportedOperationException("getLongValue() is not a valid method for result")
-  }
-
-  override def getBigDecimalValue: java.math.BigDecimal = {
-    throw new
-        UnsupportedOperationException("getBigDecimalValue() is not a valid method for result")
-  }
-
-  override def getValueObject: Object = {
-
-    result.iterator.foreach(v => getPartialFunction.update(v))
-
-    val output = getPartialFunction.eval(null)
-
-    output.asInstanceOf[Object]
-  }
-
-  override def merge(aggregator: MeasureAggregator): Unit = {
-    if (result.nonEmpty) {
-      result.iterator.foreach(v => {
-        getPartialFunction.update(v)
-      }
-      )
-
-      // clear result after submitting to partial function
-      result.clear
-    }
-
-    aggregator match {
-      case s: SparkUnknownCarbonAggregator =>
-        s.result.iterator.foreach(v => getPartialFunction.update(v))
-        s.result.clear
-      case _ => throw new Exception("Invalid merge expected type is" + this.getClass.getName);
-    }
-  }
-
-  private def getPartialFunction = {
-    if (partialFunction == null) {
-      partialFunction = partialAggregate.newInstance
-    }
-    partialFunction
-  }
-
-  override def isFirstTime: Boolean = {
-    isRowsAggregated
-  }
-
-  override def writeData(output: DataOutput): Unit = {
-    throw new UnsupportedOperationException()
-  }
-
-  override def readData(inPut: DataInput): Unit = {
-    throw new UnsupportedOperationException()
-  }
-
-  override def merge(value: Array[Byte]): Unit = {
-
-    throw new UnsupportedOperationException()
-  }
-
-  override def get(): MeasureAggregator = {
-    // Get means, Partition level aggregation is done and pending for merge with other or getValue
-    // So evaluate and store the temporary result here
-
-    this
-  }
-
-  override def compareTo(aggre: MeasureAggregator): Int = {
-    0
-  }
-
-  override def getCopy: MeasureAggregator = {
-    new SparkUnknownCarbonAggregator(partialAggregate)
-  }
-
-  override def setNewValue(newVal: Object): Unit = {
-
-  }
-
-  override def getColumns: java.util.List[ColumnExpression] = {
-    if (allColumns == null) {
-      allColumns = partialAggregate.flatMap(_ collect { case a: CarbonBoundReference => a.colExp })
-        .asJava
-    }
-    allColumns
-  }
-
-  override def agg(row: RowIntf): Unit = {
-    isRowsAggregated = true
-    val values = row.getValues.toSeq.map {
-      case s: String => org.apache.spark.unsafe.types.UTF8String.fromString(s)
-      // solve: java.math.BigDecimal cannot be cast to org.apache.spark.sql.types.Decimal
-      case d: java.math.BigDecimal =>
-        val javaDecVal = new java.math.BigDecimal(d.toString)
-        val scalaDecVal = new scala.math.BigDecimal(javaDecVal)
-        val decConverter = new org.apache.spark.sql.types.Decimal()
-
-        decConverter.set(scalaDecVal)
-      case others => others
-    }
-    result += new GenericMutableRow(values.map(a => a.asInstanceOf[Any]).toArray)
-  }
-
-  override def getNew: MeasureAggregator = {
-    new SparkUnknownCarbonAggregator()
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala
new file mode 100644
index 0000000..64930c7
--- /dev/null
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala
@@ -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.spark.sql.agg
+
+import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Expression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.catalyst.util.ArrayData
+import org.apache.spark.sql.types._
+
+case class CarbonAvgHelperExpression(
+    left: Expression,
+    right: Expression,
+    index: Int,
+    dataType: DataType)
+  extends BinaryExpression with CodegenFallback {
+
+  protected override def nullSafeEval(input1: Any, input2: Any): Any = {
+    val left = input1 match {
+      case g: ArrayData => g.getDouble(index)
+      case others => others.toString.toDouble
+    }
+    val right = input2 match {
+      case g: ArrayData => g.getDouble(index)
+      case others => others.toString.toDouble
+    }
+    if (index == 1) {
+      (left + right).toLong
+    } else {
+      dataType match {
+        case d: DecimalType =>
+        case _ => left + right
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala
new file mode 100644
index 0000000..9400b0c
--- /dev/null
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala
@@ -0,0 +1,89 @@
+/*
+ * 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.agg
+
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.types._
+
+@ExpressionDescription(
+  usage = "_FUNC_(x) - Returns the mean calculated from values of a group.")
+case class CarbonAverage(child: Expression) extends DeclarativeAggregate {
+
+  override def prettyName: String = "avg"
+
+  override def children: Seq[Expression] = child :: Nil
+
+  override def nullable: Boolean = true
+
+  // Return data type.
+  override def dataType: DataType = resultType
+
+  override def inputTypes: Seq[AbstractDataType] = Seq(NumericType)
+
+  override def checkInputDataTypes(): TypeCheckResult = TypeCheckResult.TypeCheckSuccess
+//    TypeUtils.checkForNumericExpr(child.dataType, "function average")
+
+  private lazy val resultType = child.dataType match {
+    case DecimalType.Fixed(p, s) =>
+      DecimalType.bounded(p + 4, s + 4)
+    case _ => DoubleType
+  }
+
+  private lazy val sumDataType = child.dataType match {
+    case _ @ DecimalType.Fixed(p, s) => DecimalType.bounded(p + 10, s)
+    case _ => DoubleType
+  }
+
+  private lazy val sum = AttributeReference("sum", sumDataType)()
+  private lazy val count = AttributeReference("count", LongType)()
+
+  override lazy val aggBufferAttributes = sum :: count :: Nil
+
+  override lazy val initialValues = Seq(
+    /* sum = */ Cast(Literal(0), sumDataType),
+    /* count = */ Literal(0L)
+  )
+
+  override lazy val updateExpressions = Seq(
+    /* sum = */
+    CarbonAvgHelperExpression(
+      sum,
+      Coalesce(child :: Cast(Literal(0), sumDataType) :: Nil), 0, sumDataType),
+    CarbonAvgHelperExpression(
+      count,
+      Coalesce(child :: Cast(Literal(0), LongType) :: Nil), 1, LongType)
+  )
+
+  override lazy val mergeExpressions = Seq(
+    /* sum = */ sum.left + sum.right,
+    /* count = */ count.left + count.right
+  )
+
+  // If all input are nulls, count will be 0 and we will get null after the division.
+  override lazy val evaluateExpression = child.dataType match {
+    case DecimalType.Fixed(p, s) =>
+      // increase the precision and scale to prevent precision loss
+      val dt = DecimalType.bounded(p + 14, s + 4)
+      Cast(Cast(sum, dt) / Cast(count, dt), resultType)
+    case _ =>
+      Cast(sum, resultType) / Cast(count, resultType)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala
new file mode 100644
index 0000000..f0e85b1
--- /dev/null
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala
@@ -0,0 +1,77 @@
+/*
+ * 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.agg
+
+import org.apache.spark.sql.catalyst.dsl.expressions._
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
+import org.apache.spark.sql.types._
+
+// scalastyle:off line.size.limit
+@ExpressionDescription(
+  usage = """_FUNC_(*) - Returns the total number of retrieved rows, including rows containing NULL values.
+    _FUNC_(expr) - Returns the number of rows for which the supplied expression is non-NULL.
+    _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.""")
+// scalastyle:on line.size.limit
+case class CarbonCount(
+    child: Expression,
+    extraChild: Option[Expression] = None) extends DeclarativeAggregate {
+
+  override def children: Seq[Expression] = child :: Nil
+
+  override def nullable: Boolean = false
+
+  // Return data type.
+  override def dataType: DataType = LongType
+
+  // Expected input data type.
+  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)
+
+  private lazy val count = AttributeReference("count", LongType, nullable = false)()
+
+  override lazy val aggBufferAttributes = count :: Nil
+
+  override lazy val initialValues = Seq(
+    /* count = */ Literal(0L)
+  )
+  private lazy val zero = Cast(Literal(0), dataType)
+
+  private val actualChild = extraChild.getOrElse(child)
+
+  override lazy val updateExpressions = {
+    if (actualChild.nullable) {
+      Seq(
+        /* sum = */
+        Coalesce(Seq(Add(Coalesce(Seq(count, zero)), Cast(actualChild, dataType)), count))
+      )
+    } else {
+      Seq(
+        /* sum = */
+        Add(Coalesce(Seq(count, zero)), Cast(actualChild, dataType))
+      )
+    }
+  }
+
+  override lazy val mergeExpressions = Seq(
+    /* count = */ count.left + count.right
+  )
+
+  override lazy val evaluateExpression = count
+
+  override def defaultResult: Option[Literal] = Option(Literal(0L))
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 6cbae27..fcc2e8a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -27,6 +27,7 @@ import scala.collection.mutable.ArrayBuffer
 import scala.language.implicitConversions
 
 import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
 import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
 import org.apache.spark.sql.hive.HiveContext
@@ -1101,15 +1102,9 @@ private[sql] case class ShowCreateTable(cm: tableModel, override val output: Seq
 
         if (sqlContext.tableNames(dbName).map(x => x.toLowerCase())
           .contains(tableName.toLowerCase())) {
-          if (dbName.nonEmpty) {
-            dataFrame = DataFrame(sqlContext,
-              sqlContext.catalog.lookupRelation(Seq(dbName, tableName)))
-          }
-          else {
-            dataFrame = DataFrame(sqlContext, sqlContext.catalog.lookupRelation(Seq(tableName)))
-          }
-        }
-        else {
+          dataFrame = DataFrame(sqlContext,
+            sqlContext.catalog.lookupRelation(TableIdentifier(tableName, Some(dbName))))
+        } else {
           LOGGER.error(s"Input source table $tableName does not exists")
           sys.error(s"Input source table $tableName does not exists")
         }
@@ -1164,7 +1159,7 @@ private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) e
 
     val relation =
       CarbonEnv.getInstance(sqlContext).carbonCatalog
-        .lookupRelation1(Option(schemaName), tableName, None)(sqlContext)
+        .lookupRelation1(Option(schemaName), tableName)(sqlContext)
         .asInstanceOf[CarbonRelation]
     if (relation == null) {
       sys.error(s"Table $schemaName.$tableName does not exist")
@@ -1243,20 +1238,20 @@ private[sql] case class CreateCube(cm: tableModel) extends RunnableCommand {
       try {
         sqlContext.sql(
           s"""CREATE TABLE $dbName.$tbName USING org.apache.spark.sql.CarbonSource""" +
-          s""" OPTIONS (cubename "$dbName.$tbName", tablePath "$cubePath") """).collect
+          s""" OPTIONS (tableName "$dbName.$tbName", tablePath "$cubePath") """).collect
       } catch {
         case e: Exception =>
 
+          val identifier: TableIdentifier = TableIdentifier(tbName, Some(dbName))
           val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
-            .lookupRelation2(Seq(dbName, tbName))(sqlContext).asInstanceOf[CarbonRelation]
+            .lookupRelation1(identifier)(sqlContext).asInstanceOf[CarbonRelation]
           if (relation != null) {
             LOGGER.audit(s"Deleting Table [$tbName] under Database [$dbName]" +
                          "as create TABLE failed")
             CarbonEnv.getInstance(sqlContext).carbonCatalog
-              .dropCube(relation.cubeMeta.partitioner.partitionCount,
+              .dropTable(relation.cubeMeta.partitioner.partitionCount,
                 relation.cubeMeta.storePath,
-                dbName,
-                tbName)(sqlContext)
+                identifier)(sqlContext)
           }
 
           LOGGER.audit(s"Table creation with Database name [$dbName] " +
@@ -1289,24 +1284,21 @@ private[sql] case class DeleteLoadsById(
 
     // validate load ids first
     validateLoadIds
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 
+    val identifier = TableIdentifier(tableName, Option(dbName))
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
-      Option(schemaName),
-      tableName,
-      None)(sqlContext).asInstanceOf[CarbonRelation]
+      identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
     if (relation == null) {
-      LOGGER.audit(s"The delete load by Id is failed. Table $schemaName.$tableName does not exist")
-      sys.error(s"Table $schemaName.$tableName does not exist")
+      LOGGER.audit(s"The delete load by Id is failed. Table $dbName.$tableName does not exist")
+      sys.error(s"Table $dbName.$tableName does not exist")
     }
 
-    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(schemaName + '_' + tableName)
+    val carbonTable = CarbonMetadata.getInstance().getCarbonTable(dbName + '_' + tableName)
 
     if (null == carbonTable) {
-      CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
-        Option(schemaName),
-        tableName,
-        None)(sqlContext).asInstanceOf[CarbonRelation]
+      CarbonEnv.getInstance(sqlContext).carbonCatalog
+        .lookupRelation1(identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
     }
     val path = carbonTable.getMetaDataFilepath
 
@@ -1359,30 +1351,24 @@ private[sql] case class DeleteLoadsByLoadDate(
   def run(sqlContext: SQLContext): Seq[Row] = {
 
     LOGGER.audit("The delete load by load date request has been received.")
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
-
-    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
-      Option(schemaName),
-      tableName,
-     None
-    )(sqlContext).asInstanceOf[CarbonRelation]
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
+      .lookupRelation1(identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
     if (relation == null) {
       LOGGER
-        .audit(s"The delete load by load date is failed. Table $schemaName.$tableName does not " +
+        .audit(s"The delete load by load date is failed. Table $dbName.$tableName does not " +
          s"exist")
-      sys.error(s"Table $schemaName.$tableName does not exist")
+      sys.error(s"Table $dbName.$tableName does not exist")
     }
 
     var carbonTable = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
-      .getCarbonTable(schemaName + '_' + tableName)
+      .getCarbonTable(dbName + '_' + tableName)
     var segmentStatusManager = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier)
 
     if (null == carbonTable) {
-      var relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
-        Option(schemaName),
-        tableName,
-        None
-      )(sqlContext).asInstanceOf[CarbonRelation]
+      var relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
+        .lookupRelation1(identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
     }
     var path = carbonTable.getMetaDataFilepath()
 
@@ -1409,19 +1395,20 @@ private[sql] case class LoadCube(
 
   def run(sqlContext: SQLContext): Seq[Row] = {
 
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val identifier = TableIdentifier(tableName, Option(dbName))
     if (isOverwriteExist) {
-      sys.error("Overwrite is not supported for carbon table with " + schemaName + "." + tableName)
+      sys.error("Overwrite is not supported for carbon table with " + dbName + "." + tableName)
     }
     if (null == org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
-      .getCarbonTable(schemaName + "_" + tableName)) {
-      logError("Data loading failed. table not found: " + schemaName + "_" + tableName)
-      LOGGER.audit("Data loading failed. table not found: " + schemaName + "_" + tableName)
-      sys.error("Data loading failed. table not found: " + schemaName + "_" + tableName)
+      .getCarbonTable(dbName + "_" + tableName)) {
+      logError("Data loading failed. table not found: " + dbName + "_" + tableName)
+      LOGGER.audit("Data loading failed. table not found: " + dbName + "_" + tableName)
+      sys.error("Data loading failed. table not found: " + dbName + "_" + tableName)
     }
     CarbonProperties.getInstance().addProperty("zookeeper.enable.lock", "false")
     val carbonLock = CarbonLockFactory.getCarbonLockObj(org.carbondata.core.
-      carbon.metadata.CarbonMetadata.getInstance().getCarbonTable(schemaName + "_" + tableName).
+      carbon.metadata.CarbonMetadata.getInstance().getCarbonTable(dbName + "_" + tableName).
       getMetaDataFilepath, LockUsage.METADATA_LOCK)
     try {
       if (carbonLock.lockWithRetries()) {
@@ -1432,12 +1419,10 @@ private[sql] case class LoadCube(
       }
 
       val factPath = FileUtils.getPaths(CarbonUtil.checkAndAppendHDFSUrl(factPathFromUser))
-      val relation =
-        CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .lookupRelation1(Option(schemaName), tableName, None)(sqlContext)
-          .asInstanceOf[CarbonRelation]
+      val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
+        .lookupRelation1(identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
       if (relation == null) {
-        sys.error(s"Table $schemaName.$tableName does not exist")
+        sys.error(s"Table $dbName.$tableName does not exist")
       }
       val carbonLoadModel = new CarbonLoadModel()
       carbonLoadModel.setTableName(relation.cubeMeta.carbonTableIdentifier.getTableName)
@@ -1511,7 +1496,7 @@ private[sql] case class LoadCube(
         if (null == relation.cubeMeta.partitioner.partitionColumn ||
             relation.cubeMeta.partitioner.partitionColumn(0).isEmpty) {
           LOGGER.info("Initiating Direct Load for the Table : (" +
-                      schemaName + "." + tableName + ")")
+                      dbName + "." + tableName + ")")
           carbonLoadModel.setFactFilePath(factPath)
           carbonLoadModel.setCsvDelimiter(CarbonUtil.unescapeChar(delimiter))
           carbonLoadModel.setCsvHeader(fileHeader)
@@ -1526,9 +1511,9 @@ private[sql] case class LoadCube(
           partitionLocation += System.currentTimeMillis()
           FileFactory.mkdirs(partitionLocation, fileType)
           LOGGER.info("Initiating Data Partitioning for the Table : (" +
-                      schemaName + "." + tableName + ")")
+                      dbName + "." + tableName + ")")
           partitionStatus = CarbonContext.partitionData(
-            schemaName,
+            dbName,
             tableName,
             factPath,
             partitionLocation,
@@ -1605,8 +1590,9 @@ private[sql] case class PartitionData(databaseName: String, tableName: String, f
   var partitionStatus = CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS
 
   def run(sqlContext: SQLContext): Seq[Row] = {
-    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
-      Option(databaseName), tableName, None)(sqlContext).asInstanceOf[CarbonRelation]
+    val identifier = TableIdentifier(tableName, Option(databaseName))
+    val relation = CarbonEnv.getInstance(sqlContext)
+      .carbonCatalog.lookupRelation1(identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
     val dimNames = relation.cubeMeta.carbonTable
       .getDimensionByTableName(tableName).asScala.map(_.getColName)
     val msrNames = relation.cubeMeta.carbonTable
@@ -1624,60 +1610,18 @@ private[sql] case class PartitionData(databaseName: String, tableName: String, f
   }
 }
 
-private[sql] case class LoadAggregationTable(
-    newSchema: CarbonTable,
-    schemaName: String,
-    cubeName: String,
-    aggTableName: String) extends RunnableCommand {
-
-  def run(sqlContext: SQLContext): Seq[Row] = {
-    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
-      Option(schemaName),
-      cubeName,
-      None)(sqlContext).asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      sys.error(s"Table $schemaName.$cubeName does not exist")
-    }
-    val carbonLoadModel = new CarbonLoadModel()
-    carbonLoadModel.setTableName(cubeName)
-    val table = relation.cubeMeta.carbonTable
-    carbonLoadModel.setAggTableName(aggTableName)
-    carbonLoadModel.setTableName(table.getFactTableName)
-    carbonLoadModel.setAggLoadRequest(true)
-    var storeLocation = CarbonProperties.getInstance
-      .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
-        System.getProperty("java.io.tmpdir"))
-    storeLocation = storeLocation + "/carbonstore/" + System.currentTimeMillis()
-    val columinar = sqlContext.getConf("carbon.is.columnar.storage", "true").toBoolean
-    var kettleHomePath = sqlContext.getConf("carbon.kettle.home", null)
-    if (null == kettleHomePath) {
-      kettleHomePath = CarbonProperties.getInstance.getProperty("carbon.kettle.home")
-    }
-    if (kettleHomePath == null) {
-      sys.error(s"carbon.kettle.home is not set")
-    }
-    CarbonDataRDDFactory.loadCarbonData(
-      sqlContext,
-      carbonLoadModel,
-      storeLocation,
-      relation.cubeMeta.storePath,
-      kettleHomePath,
-      relation.cubeMeta.partitioner, columinar, isAgg = true)
-    Seq.empty
-  }
-}
-
-
 private[sql] case class ShowAllTablesInSchema(
     schemaNameOp: Option[String],
     override val output: Seq[Attribute]
 ) extends RunnableCommand {
 
   override def run(sqlContext: SQLContext): Seq[Row] = {
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
-    CarbonEnv.getInstance(sqlContext).carbonCatalog.getCubes(Some(schemaName))(sqlContext).map(
-      x => Row(x._1,
-        sqlContext.asInstanceOf[HiveContext].catalog.tableExists(Seq(schemaName, x._1))))
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    CarbonEnv.getInstance(sqlContext).carbonCatalog.getTables(Some(dbName))(sqlContext)
+      .map{x =>
+        Row(x._1, sqlContext.asInstanceOf[HiveContext]
+          .catalog.tableExists(TableIdentifier(x._1, Some(dbName))))
+      }
   }
 }
 
@@ -1685,9 +1629,9 @@ private[sql] case class ShowAllTables(override val output: Seq[Attribute])
   extends RunnableCommand {
 
   override def run(sqlContext: SQLContext): Seq[Row] = {
-    CarbonEnv.getInstance(sqlContext).carbonCatalog.getAllCubes()(sqlContext)
+    CarbonEnv.getInstance(sqlContext).carbonCatalog.getAllTables()(sqlContext)
       .map { x =>
-        Row(x._1, x._2, sqlContext.asInstanceOf[HiveContext].catalog.tableExists(Seq(x._1, x._2)))
+        Row(x.database.get, x.table, sqlContext.asInstanceOf[HiveContext].catalog.tableExists(x))
       }
   }
 
@@ -1705,18 +1649,19 @@ private[sql] case class ShowAllTablesDetail(
   }
 }
 
-private[sql] case class MergeTable(schemaName: String, cubeName: String, tableName: String)
+private[sql] case class MergeTable(dbName: String, cubeName: String, tableName: String)
   extends RunnableCommand {
 
   def run(sqlContext: SQLContext): Seq[Row] = {
+    val identifier = TableIdentifier(tableName, Option(cubeName))
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
-      .lookupRelation2(Seq(schemaName, cubeName), None)(sqlContext).asInstanceOf[CarbonRelation]
+      .lookupRelation1(identifier, None)(sqlContext).asInstanceOf[CarbonRelation]
     if (relation == null) {
-      sys.error(s"Table $schemaName.$cubeName does not exist")
+      sys.error(s"Table $dbName.$cubeName does not exist")
     }
     val carbonLoadModel = new CarbonLoadModel()
     carbonLoadModel.setTableName(cubeName)
-    carbonLoadModel.setDatabaseName(schemaName)
+    carbonLoadModel.setDatabaseName(dbName)
     val table = relation.cubeMeta.carbonTable
     var isTablePresent = false
     if (table.getFactTableName.equals(tableName)) {
@@ -1749,34 +1694,35 @@ private[sql] case class MergeTable(schemaName: String, cubeName: String, tableNa
 }
 
 private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Option[String],
-    cubeName: String)
+    tableName: String)
   extends RunnableCommand {
 
   def run(sqlContext: SQLContext): Seq[Row] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val identifier = TableIdentifier(tableName, Option(dbName))
     val tmpTable = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
-      .getCarbonTable(schemaName + "_" + cubeName)
+      .getCarbonTable(dbName + "_" + tableName)
     if (null == tmpTable) {
       if (!ifExistsSet) {
         LOGGER
-          .audit(s"Dropping carbon table with Database name [$schemaName] and Table name" +
+          .audit(s"Dropping carbon table with Database name [$dbName] and Table name" +
                  "[$cubeName] failed")
-        LOGGER.error(s"Carbon Table $schemaName.$cubeName metadata does not exist")
+        LOGGER.error(s"Carbon Table $dbName.$tableName metadata does not exist")
       }
-      if (sqlContext.tableNames(schemaName).map(x => x.toLowerCase())
-        .contains(cubeName.toLowerCase())) {
+      if (sqlContext.tableNames(dbName).map(x => x.toLowerCase())
+        .contains(tableName.toLowerCase())) {
         try {
           sqlContext.asInstanceOf[HiveContext].catalog.client.
-            runSqlHive(s"DROP TABLE IF EXISTS $schemaName.$cubeName")
+            runSqlHive(s"DROP TABLE IF EXISTS $dbName.$tableName")
         } catch {
           case e: RuntimeException =>
             LOGGER.audit(
-              s"Error While deleting the table $schemaName.$cubeName during drop carbon table" +
+              s"Error While deleting the table $dbName.$tableName during drop carbon table" +
               e.getMessage)
         }
       } else if (!ifExistsSet) {
-        sys.error(s"Carbon Table $schemaName.$cubeName does not exist")
+        sys.error(s"Carbon Table $dbName.$tableName does not exist")
       }
     } else {
       CarbonProperties.getInstance().addProperty("zookeeper.enable.lock", "false")
@@ -1787,32 +1733,33 @@ private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Opti
           logInfo("Successfully able to get the table metadata file lock")
         } else {
           LOGGER.audit(
-            s"Dropping table with Database name [$schemaName] and Table name [$cubeName] " +
+            s"Dropping table with Database name [$dbName] and Table name [$tableName] " +
             s"failed as the Table is locked")
           sys.error("Table is locked for updation. Please try after some time")
         }
 
         val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .lookupRelation2(Seq(schemaName, cubeName))(sqlContext).asInstanceOf[CarbonRelation]
+          .lookupRelation1(identifier)(sqlContext).asInstanceOf[CarbonRelation]
 
         if (relation == null) {
           if (!ifExistsSet) {
-            sys.error(s"Table $schemaName.$cubeName does not exist")
+            sys.error(s"Table $dbName.$tableName does not exist")
           }
         } else {
-          LOGGER.audit(s"Deleting table [$cubeName] under database [$schemaName]")
+          LOGGER.audit(s"Deleting table [$tableName] under database [$dbName]")
 
           CarbonEnv.getInstance(sqlContext).carbonCatalog
-            .dropCube(relation.cubeMeta.partitioner.partitionCount,
+            .dropTable(relation.cubeMeta.partitioner.partitionCount,
               relation.cubeMeta.storePath,
-              relation.cubeMeta.carbonTableIdentifier.getDatabaseName,
-              relation.cubeMeta.carbonTableIdentifier.getTableName)(sqlContext)
+              TableIdentifier(relation.cubeMeta.carbonTableIdentifier.getTableName,
+                Some(relation.cubeMeta.carbonTableIdentifier.getDatabaseName))
+              )(sqlContext)
           CarbonDataRDDFactory
-            .dropCube(sqlContext.sparkContext, schemaName, cubeName,
+            .dropCube(sqlContext.sparkContext, dbName, tableName,
               relation.cubeMeta.partitioner)
-          QueryPartitionHelper.getInstance().removePartition(schemaName, cubeName)
+          QueryPartitionHelper.getInstance().removePartition(dbName, tableName)
 
-          LOGGER.audit(s"Deleted table [$cubeName] under database [$schemaName]")
+          LOGGER.audit(s"Deleted table [$tableName] under database [$dbName]")
         }
       }
       finally {
@@ -1825,7 +1772,7 @@ private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Opti
               CarbonUtil.deleteFoldersAndFiles(file.getParentFile)
             }
             // delete bad record log after drop cube
-            val badLogPath = CarbonUtil.getBadLogPath(schemaName +  File.separator + cubeName)
+            val badLogPath = CarbonUtil.getBadLogPath(dbName +  File.separator + tableName)
             val badLogFileType = FileFactory.getFileType(badLogPath)
             if (FileFactory.isFileExist(badLogPath, badLogFileType)) {
               val file = FileFactory.getCarbonFile(badLogPath, badLogFileType)
@@ -1847,19 +1794,20 @@ private[sql] case class DropAggregateTableCommand(ifExistsSet: Boolean,
     tableName: String) extends RunnableCommand {
 
   def run(sqlContext: SQLContext): Seq[Row] = {
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
-    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.
-      lookupRelation1(Some(schemaName), tableName, None)(sqlContext).asInstanceOf[CarbonRelation]
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val identifier = TableIdentifier(tableName, Option(dbName))
+    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
+      .lookupRelation1(identifier)(sqlContext).asInstanceOf[CarbonRelation]
 
     if (relation == null) {
       if (!ifExistsSet) {
-        sys.error(s"Aggregate Table $schemaName.$tableName does not exist")
+        sys.error(s"Aggregate Table $dbName.$tableName does not exist")
       }
     }
     else {
       CarbonDataRDDFactory.dropAggregateTable(
         sqlContext.sparkContext,
-        schemaName,
+        dbName,
         tableName,
         relation.cubeMeta.partitioner)
     }
@@ -1941,12 +1889,12 @@ private[sql] case class ShowAggregateTables(
 private[sql] case class DescribeCommandFormatted(
     child: SparkPlan,
     override val output: Seq[Attribute],
-    tblIdentifier: Seq[String])
+    tblIdentifier: TableIdentifier)
   extends RunnableCommand {
 
   override def run(sqlContext: SQLContext): Seq[Row] = {
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
-      .lookupRelation2(tblIdentifier, None)(sqlContext).asInstanceOf[CarbonRelation]
+      .lookupRelation1(tblIdentifier)(sqlContext).asInstanceOf[CarbonRelation]
     var results: Seq[(String, String, String)] = child.schema.fields.map { field =>
       val comment = if (relation.metaData.dims.contains(field.name)) {
         val dimension = relation.metaData.carbonTable.getDimensionByName(
@@ -2036,15 +1984,16 @@ private[sql] case class DeleteLoadByDate(
   def run(sqlContext: SQLContext): Seq[Row] = {
 
     LOGGER.audit("The delete load by date request has been received.")
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val identifier = TableIdentifier(cubeName, Option(dbName))
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
-      .lookupRelation1(Some(schemaName), cubeName, None)(sqlContext).asInstanceOf[CarbonRelation]
+      .lookupRelation1(identifier)(sqlContext).asInstanceOf[CarbonRelation]
     var level: String = ""
     var carbonTable = org.carbondata.core.carbon.metadata.CarbonMetadata
-         .getInstance().getCarbonTable(schemaName + '_' + cubeName)
+         .getInstance().getCarbonTable(dbName + '_' + cubeName)
     if (relation == null) {
-      LOGGER.audit(s"The delete load by date is failed. Table $schemaName.$cubeName does not exist")
-      sys.error(s"Table $schemaName.$cubeName does not exist")
+      LOGGER.audit(s"The delete load by date is failed. Table $dbName.$cubeName does not exist")
+      sys.error(s"Table $dbName.$cubeName does not exist")
     }
 
     val matches: Seq[AttributeReference] = relation.dimensionsAttr.filter(
@@ -2054,8 +2003,8 @@ private[sql] case class DeleteLoadByDate(
     if (matches.isEmpty) {
       LOGGER.audit(
         "The delete load by date is failed. " +
-        "Table $schemaName.$cubeName does not contain date field " + dateField)
-      sys.error(s"Table $schemaName.$cubeName does not contain date field " + dateField)
+        "Table $dbName.$cubeName does not contain date field " + dateField)
+      sys.error(s"Table $dbName.$cubeName does not contain date field " + dateField)
     }
     else {
       level = matches.asJava.get(0).name
@@ -2067,7 +2016,7 @@ private[sql] case class DeleteLoadByDate(
     CarbonDataRDDFactory.deleteLoadByDate(
       sqlContext,
       new CarbonDataLoadSchema(carbonTable),
-      schemaName,
+      dbName,
       cubeName,
       tableName,
       CarbonEnv.getInstance(sqlContext).carbonCatalog.storePath,
@@ -2089,13 +2038,14 @@ private[sql] case class CleanFiles(
 
   def run(sqlContext: SQLContext): Seq[Row] = {
     LOGGER.audit("The clean files request has been received.")
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    val identifier = TableIdentifier(cubeName, Option(dbName))
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
-      .lookupRelation1(Some(schemaName), cubeName, None)(sqlContext).
+      .lookupRelation1(identifier)(sqlContext).
       asInstanceOf[CarbonRelation]
     if (relation == null) {
-      LOGGER.audit(s"The clean files request is failed. Table $schemaName.$cubeName does not exist")
-      sys.error(s"Table $schemaName.$cubeName does not exist")
+      LOGGER.audit(s"The clean files request is failed. Table $dbName.$cubeName does not exist")
+      sys.error(s"Table $dbName.$cubeName does not exist")
     }
 
     val carbonLoadModel = new CarbonLoadModel()


[21/56] [abbrv] incubator-carbondata git commit: remove AbastructQueryExecutor (#690)

Posted by jb...@apache.org.
remove AbastructQueryExecutor (#690)



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

Branch: refs/heads/master
Commit: 656577d5c05d57291de77084d8993553a49cdfdc
Parents: 6288ec7
Author: Jacky Li <ja...@huawei.com>
Authored: Sat Jun 18 14:41:02 2016 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sat Jun 18 12:11:02 2016 +0530

----------------------------------------------------------------------
 .../impl/ListBasedResultAggregator.java         |  10 +-
 .../impl/InternalAbstractQueryExecutor.java     | 120 -------------------
 2 files changed, 4 insertions(+), 126 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/656577d5/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
index 074e9ea..10bf88c 100644
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
@@ -177,13 +177,11 @@ public class ListBasedResultAggregator implements ScannedResultAggregator {
    * Below method will used to get the result
    */
   @Override public Result getAggregatedResult() {
-    Result result = new ListBasedResult();
+    Result<List<ListBasedResultWrapper>, Object> result = new ListBasedResult();
     if (!tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
-      updateKeyWithLatestBlockKeygenerator();
-      result.addScannedResult(listBasedResult);
-    } else {
-      result.addScannedResult(listBasedResult);
+      updateKeyWithLatestBlockKeyGenerator();
     }
+    result.addScannedResult(listBasedResult);
     return result;
   }
 
@@ -195,7 +193,7 @@ public class ListBasedResultAggregator implements ScannedResultAggregator {
    *
    * @return updated block
    */
-  private void updateKeyWithLatestBlockKeygenerator() {
+  private void updateKeyWithLatestBlockKeyGenerator() {
     try {
       long[] data = null;
       ByteArrayWrapper key = null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/656577d5/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
deleted file mode 100644
index a8c2c3a..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
+++ /dev/null
@@ -1,120 +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.carbondata.query.carbon.executor.internal.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.carbon.datastore.DataRefNodeFinder;
-import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.merger.ScannedResultMerger;
-import org.carbondata.query.carbon.merger.impl.UnSortedScannedResultMerger;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Query executor class which will have common implementation
- * among all type of executor
- */
-public abstract class InternalAbstractQueryExecutor implements InternalQueryExecutor {
-
-  /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(InternalAbstractQueryExecutor.class.getName());
-
-  /**
-   * Executor Service.
-   */
-  protected ExecutorService execService;
-
-  /**
-   * number of cores to be used
-   */
-  protected int numberOfCores;
-
-  /**
-   * Below method will be used to used to execute the detail query
-   * and it will return iterator over result
-   *
-   * @param tableBlockExecutionInfosList block execution info which will have all the properties
-   *                       required for query execution
-   * @param sliceIndex   slice indexes to be executed in this case it w
-   * @return query result
-   */
-  @Override public CarbonIterator<Result> executeQuery(
-      List<BlockExecutionInfo> tableBlockExecutionInfosList, int[] sliceIndex)
-      throws QueryExecutionException {
-
-    long startTime = System.currentTimeMillis();
-    BlockExecutionInfo latestInfo =
-        tableBlockExecutionInfosList.get(tableBlockExecutionInfosList.size() - 1);
-    execService = Executors.newFixedThreadPool(numberOfCores);
-    ScannedResultMerger scannedResultProcessor = null;
-    scannedResultProcessor = new UnSortedScannedResultMerger(latestInfo, numberOfCores);
-    try {
-      List<Future> listFutureObjects = new ArrayList<Future>();
-      for (BlockExecutionInfo blockInfo : tableBlockExecutionInfosList) {
-        DataRefNodeFinder finder = new BTreeDataRefNodeFinder(blockInfo.getEachColumnValueSize());
-        DataRefNode startDataBlock = finder
-            .findFirstDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getStartKey());
-        DataRefNode endDataBlock = finder
-            .findLastDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getEndKey());
-        long numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
-        blockInfo.setFirstDataBlock(startDataBlock);
-        blockInfo.setNumberOfBlockToScan(numberOfBlockToScan);
-        blockInfo.setScannedResultProcessor(scannedResultProcessor);
-        listFutureObjects.add(execService.submit(new QueryRunner(blockInfo)));
-      }
-      execService.shutdown();
-      execService.awaitTermination(2, TimeUnit.DAYS);
-      LOGGER.info("Total time taken for scan " + (System.currentTimeMillis() - startTime));
-      for (Future future : listFutureObjects) {
-        try {
-          future.get();
-        } catch (ExecutionException e) {
-          throw new QueryExecutionException(e.getMessage());
-        }
-      }
-      return scannedResultProcessor.getQueryResultIterator();
-    } catch (QueryExecutionException e) {
-      LOGGER.error(e, e.getMessage());
-      throw new QueryExecutionException(e);
-    } catch (InterruptedException e) {
-      LOGGER.error(e, e.getMessage());
-      throw new QueryExecutionException(e);
-    } finally {
-      execService = null;
-      latestInfo = null;
-    }
-  }
-
-}


[41/56] [abbrv] incubator-carbondata git commit: Supported both Spark 1.5.2 and 1.6.1 versions in Carbondata (#693)

Posted by jb...@apache.org.
Supported both Spark 1.5.2 and 1.6.1 versions in Carbondata (#693)

* Supported both Spark 1.5.2 and 1.6.1 versions in Carbon

* Fixed bug

* Changed default version to 1.5.2


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

Branch: refs/heads/master
Commit: 19590dba38f69c262c8865e012e9620f71a3f0ab
Parents: 1c725f5
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Sun Jun 19 14:34:16 2016 +0530
Committer: Jacky Li <ja...@huawei.com>
Committed: Sun Jun 19 17:04:16 2016 +0800

----------------------------------------------------------------------
 .../spark/sql/common/util/QueryTest.scala       | 16 ----
 .../spark/sql/CarbonDatasourceRelation.scala    |  2 +-
 .../org/apache/spark/sql/CarbonOperators.scala  |  8 +-
 .../sql/agg/CarbonAggregationExpression.scala   | 50 -----------
 .../apache/spark/sql/agg/CarbonAverage.scala    | 89 --------------------
 .../org/apache/spark/sql/agg/CarbonCount.scala  | 77 -----------------
 .../spark/sql/catalyst/TableIdentifier.scala    | 57 +++++++++++++
 .../spark/sql/hive/CarbonMetastoreCatalog.scala | 24 ++----
 .../spark/sql/hive/CarbonSQLDialect.scala       |  6 +-
 .../spark/sql/hive/CarbonStrategies.scala       |  6 +-
 .../spark/sql/optimizer/CarbonOptimizer.scala   |  2 +-
 .../spark/sql/common/util/QueryTest.scala       | 16 ----
 pom.xml                                         | 15 +++-
 13 files changed, 93 insertions(+), 275 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala b/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
index c4f09cc..f9960d3 100644
--- a/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
+++ b/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
@@ -23,7 +23,6 @@ import scala.collection.JavaConversions._
 
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.execution.columnar.InMemoryRelation
 import org.apache.spark.sql.{DataFrame, Row, SQLContext}
 
 
@@ -71,21 +70,6 @@ class QueryTest extends PlanTest {
     }
   }
 
-  /**
-   * Asserts that a given [[DataFrame]] will be executed using the given number of cached results.
-   */
-  def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = {
-    val planWithCaching = query.queryExecution.withCachedData
-    val cachedData = planWithCaching collect {
-      case cached: InMemoryRelation => cached
-    }
-
-    assert(
-      cachedData.size == numCachedTables,
-      s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" +
-        planWithCaching)
-  }
-
   protected def checkAnswer(df: DataFrame, expectedAnswer: Row): Unit = {
     checkAnswer(df, Seq(expectedAnswer))
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
index 94b38a4..e14f86a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
@@ -58,7 +58,7 @@ class CarbonSource
         val options = new CarbonOption(parameters)
         val tableIdentifier = options.tableIdentifier.split("""\.""").toSeq
         val ident = tableIdentifier match {
-          case Seq(name) => TableIdentifier(name)
+          case Seq(name) => TableIdentifier(name, None)
           case Seq(db, name) => TableIdentifier(name, Some(db))
         }
         CarbonDatasourceRelation(ident, None)(sqlContext)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index 6b2bdc2..782b87c 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -39,7 +39,8 @@ import org.carbondata.spark.rdd.CarbonScanRDD
 case class CarbonScan(
     var attributesRaw: Seq[Attribute],
     relationRaw: CarbonRelation,
-    dimensionPredicatesRaw: Seq[Expression])(@transient val ocRaw: SQLContext) extends LeafNode {
+    dimensionPredicatesRaw: Seq[Expression],
+    useUnsafeCoversion: Boolean = true)(@transient val ocRaw: SQLContext) extends LeafNode {
   val carbonTable = relationRaw.metaData.carbonTable
   val selectedDims = scala.collection.mutable.MutableList[QueryDimension]()
   val selectedMsrs = scala.collection.mutable.MutableList[QueryMeasure]()
@@ -184,10 +185,11 @@ case class CarbonScan(
   }
 
 
-  override def outputsUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
+  override def outputsUnsafeRows: Boolean =
+    (attributesNeedToDecode.size() == 0) && useUnsafeCoversion
 
   override def doExecute(): RDD[InternalRow] = {
-    val outUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
+    val outUnsafeRows: Boolean = (attributesNeedToDecode.size() == 0) && useUnsafeCoversion
     inputRdd.mapPartitions { iter =>
       val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
       new Iterator[InternalRow] {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala
deleted file mode 100644
index 64930c7..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAggregationExpression.scala
+++ /dev/null
@@ -1,50 +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.agg
-
-import org.apache.spark.sql.catalyst.expressions.{BinaryExpression, Expression}
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
-import org.apache.spark.sql.catalyst.util.ArrayData
-import org.apache.spark.sql.types._
-
-case class CarbonAvgHelperExpression(
-    left: Expression,
-    right: Expression,
-    index: Int,
-    dataType: DataType)
-  extends BinaryExpression with CodegenFallback {
-
-  protected override def nullSafeEval(input1: Any, input2: Any): Any = {
-    val left = input1 match {
-      case g: ArrayData => g.getDouble(index)
-      case others => others.toString.toDouble
-    }
-    val right = input2 match {
-      case g: ArrayData => g.getDouble(index)
-      case others => others.toString.toDouble
-    }
-    if (index == 1) {
-      (left + right).toLong
-    } else {
-      dataType match {
-        case d: DecimalType =>
-        case _ => left + right
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala
deleted file mode 100644
index 9400b0c..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonAverage.scala
+++ /dev/null
@@ -1,89 +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.agg
-
-import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
-import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
-import org.apache.spark.sql.types._
-
-@ExpressionDescription(
-  usage = "_FUNC_(x) - Returns the mean calculated from values of a group.")
-case class CarbonAverage(child: Expression) extends DeclarativeAggregate {
-
-  override def prettyName: String = "avg"
-
-  override def children: Seq[Expression] = child :: Nil
-
-  override def nullable: Boolean = true
-
-  // Return data type.
-  override def dataType: DataType = resultType
-
-  override def inputTypes: Seq[AbstractDataType] = Seq(NumericType)
-
-  override def checkInputDataTypes(): TypeCheckResult = TypeCheckResult.TypeCheckSuccess
-//    TypeUtils.checkForNumericExpr(child.dataType, "function average")
-
-  private lazy val resultType = child.dataType match {
-    case DecimalType.Fixed(p, s) =>
-      DecimalType.bounded(p + 4, s + 4)
-    case _ => DoubleType
-  }
-
-  private lazy val sumDataType = child.dataType match {
-    case _ @ DecimalType.Fixed(p, s) => DecimalType.bounded(p + 10, s)
-    case _ => DoubleType
-  }
-
-  private lazy val sum = AttributeReference("sum", sumDataType)()
-  private lazy val count = AttributeReference("count", LongType)()
-
-  override lazy val aggBufferAttributes = sum :: count :: Nil
-
-  override lazy val initialValues = Seq(
-    /* sum = */ Cast(Literal(0), sumDataType),
-    /* count = */ Literal(0L)
-  )
-
-  override lazy val updateExpressions = Seq(
-    /* sum = */
-    CarbonAvgHelperExpression(
-      sum,
-      Coalesce(child :: Cast(Literal(0), sumDataType) :: Nil), 0, sumDataType),
-    CarbonAvgHelperExpression(
-      count,
-      Coalesce(child :: Cast(Literal(0), LongType) :: Nil), 1, LongType)
-  )
-
-  override lazy val mergeExpressions = Seq(
-    /* sum = */ sum.left + sum.right,
-    /* count = */ count.left + count.right
-  )
-
-  // If all input are nulls, count will be 0 and we will get null after the division.
-  override lazy val evaluateExpression = child.dataType match {
-    case DecimalType.Fixed(p, s) =>
-      // increase the precision and scale to prevent precision loss
-      val dt = DecimalType.bounded(p + 14, s + 4)
-      Cast(Cast(sum, dt) / Cast(count, dt), resultType)
-    case _ =>
-      Cast(sum, resultType) / Cast(count, resultType)
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala b/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala
deleted file mode 100644
index f0e85b1..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/agg/CarbonCount.scala
+++ /dev/null
@@ -1,77 +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.agg
-
-import org.apache.spark.sql.catalyst.dsl.expressions._
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.DeclarativeAggregate
-import org.apache.spark.sql.types._
-
-// scalastyle:off line.size.limit
-@ExpressionDescription(
-  usage = """_FUNC_(*) - Returns the total number of retrieved rows, including rows containing NULL values.
-    _FUNC_(expr) - Returns the number of rows for which the supplied expression is non-NULL.
-    _FUNC_(DISTINCT expr[, expr...]) - Returns the number of rows for which the supplied expression(s) are unique and non-NULL.""")
-// scalastyle:on line.size.limit
-case class CarbonCount(
-    child: Expression,
-    extraChild: Option[Expression] = None) extends DeclarativeAggregate {
-
-  override def children: Seq[Expression] = child :: Nil
-
-  override def nullable: Boolean = false
-
-  // Return data type.
-  override def dataType: DataType = LongType
-
-  // Expected input data type.
-  override def inputTypes: Seq[AbstractDataType] = Seq(AnyDataType)
-
-  private lazy val count = AttributeReference("count", LongType, nullable = false)()
-
-  override lazy val aggBufferAttributes = count :: Nil
-
-  override lazy val initialValues = Seq(
-    /* count = */ Literal(0L)
-  )
-  private lazy val zero = Cast(Literal(0), dataType)
-
-  private val actualChild = extraChild.getOrElse(child)
-
-  override lazy val updateExpressions = {
-    if (actualChild.nullable) {
-      Seq(
-        /* sum = */
-        Coalesce(Seq(Add(Coalesce(Seq(count, zero)), Cast(actualChild, dataType)), count))
-      )
-    } else {
-      Seq(
-        /* sum = */
-        Add(Coalesce(Seq(count, zero)), Cast(actualChild, dataType))
-      )
-    }
-  }
-
-  override lazy val mergeExpressions = Seq(
-    /* count = */ count.left + count.right
-  )
-
-  override lazy val evaluateExpression = count
-
-  override def defaultResult: Option[Literal] = Option(Literal(0L))
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala b/integration/spark/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala
new file mode 100644
index 0000000..b5b6eff
--- /dev/null
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/catalyst/TableIdentifier.scala
@@ -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.spark.sql.catalyst
+
+import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
+
+/**
+ * Identifies a `table` in `database`.  If `database` is not defined, the current database is used.
+ */
+private[sql] case class TableIdentifier(table: String, database: Option[String]) {
+  def this(table: String) = this(table, None)
+
+  override def toString: String = quotedString
+
+  def quotedString: String = database.map(db => s"`$db`.`$table`").getOrElse(s"`$table`")
+
+  def unquotedString: String = database.map(db => s"$db.$table").getOrElse(table)
+
+  def withDatabase(database: String): TableIdentifier = this.copy(database = Some(database))
+
+  def toSeq: Seq[String] = database.toSeq :+ table
+
+}
+
+private[sql] object TableIdentifier {
+  def apply(tableName: String): TableIdentifier = new TableIdentifier(tableName)
+
+  implicit def toTableIdentifier(tableIdentifier: Seq[String]): TableIdentifier = {
+    tableIdentifier match {
+      case Seq(dbName, tableName) => TableIdentifier(tableName, Some(dbName))
+      case Seq(tableName) => TableIdentifier(tableName, None)
+      case _ => throw new NoSuchTableException
+    }
+  }
+
+  implicit def toSequence(tableIdentifier: TableIdentifier): Seq[String] = {
+    tableIdentifier.database match {
+      case Some(dbName) => Seq(dbName, tableIdentifier.table)
+      case _ => Seq(tableIdentifier.table)
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
index 30a1070..9f1a2d6 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
@@ -68,7 +68,7 @@ object CarbonMetastoreCatalog {
   def readSchemaFileToThriftTable(schemaFilePath: String): TableInfo = {
     val createTBase = new ThriftReader.TBaseCreator() {
       override def create(): org.apache.thrift.TBase[TableInfo, TableInfo._Fields] = {
-        return new TableInfo();
+        new TableInfo()
       }
     }
     val thriftReader = new ThriftReader(schemaFilePath, createTBase)
@@ -85,10 +85,10 @@ object CarbonMetastoreCatalog {
   def writeThriftTableToSchemaFile(schemaFilePath: String, tableInfo: TableInfo): Unit = {
     val thriftWriter = new ThriftWriter(schemaFilePath, false)
     try {
-      thriftWriter.open();
+      thriftWriter.open()
       thriftWriter.write(tableInfo);
     } finally {
-      thriftWriter.close();
+      thriftWriter.close()
     }
   }
 
@@ -121,16 +121,6 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     false
   }
 
-  override def lookupRelation(tableIdentifier: TableIdentifier,
-      alias: Option[String] = None): LogicalPlan = {
-    try {
-      super.lookupRelation(tableIdentifier, alias)
-    } catch {
-      case s: java.lang.Exception =>
-        lookupRelation1(tableIdentifier, alias)(hive.asInstanceOf[SQLContext])
-    }
-  }
-
   def getCubeCreationTime(schemaName: String, cubeName: String): Long = {
     val cubeMeta = metadata.cubesMeta.filter(
       c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(schemaName) &&
@@ -253,7 +243,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
                     .setMetaDataFilepath(CarbonTablePath.getFolderContainingFile(schemaFilePath))
                   CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
                   val carbonTable = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
-                      .getCarbonTable(cubeUniqueName);
+                      .getCarbonTable(cubeUniqueName)
                   metaDataBuffer += TableMeta(
                     carbonTable.getCarbonTableIdentifier,
                     storePath,
@@ -306,7 +296,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
       .add(schemaEvolutionEntry)
 
     val carbonTableIdentifier = new CarbonTableIdentifier(dbName, tableName,
-        tableInfo.getFactTable().getTableId())
+        tableInfo.getFactTable.getTableId)
     val carbonTablePath = CarbonStorePath.getCarbonTablePath(storePath, carbonTableIdentifier)
     val schemaFilePath = carbonTablePath.getSchemaFilePath
     val schemaMetadataPath = CarbonTablePath.getFolderContainingFile(schemaFilePath)
@@ -344,7 +334,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     CarbonMetadata.getInstance().loadTableMetadata(wrapperTableInfo)
     val carbonTable = CarbonMetadata.getInstance().getCarbonTable(
       wrapperTableInfo.getTableUniqueName)
-    for (i <- 0 until metadata.cubesMeta.size) {
+    for (i <- metadata.cubesMeta.indices) {
       if (wrapperTableInfo.getTableUniqueName.equals(
         metadata.cubesMeta(i).carbonTableIdentifier.getTableUniqueName)) {
         metadata.cubesMeta(i).carbonTable = carbonTable
@@ -508,7 +498,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     val dbName = tableIdentifier.database.get
     val tableName = tableIdentifier.table
     if (!tableExists(tableIdentifier)(sqlContext)) {
-      LOGGER.audit(s"Drop Table failed. Table with ${dbName}.$tableName does not exist")
+      LOGGER.audit(s"Drop Table failed. Table with $dbName.$tableName does not exist")
       sys.error(s"Table with $dbName.$tableName does not exist")
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
index edfaa90..d80703e 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
@@ -18,11 +18,12 @@
 package org.apache.spark.sql.hive
 
 import org.apache.spark.sql.CarbonSqlParser
+import org.apache.spark.sql.catalyst.ParserDialect
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 
-private[spark] class CarbonSQLDialect(context: HiveContext) extends HiveQLDialect(context) {
+private[spark] class CarbonSQLDialect(context: HiveContext) extends ParserDialect {
 
   @transient
   protected val sqlParser = new CarbonSqlParser
@@ -36,7 +37,8 @@ private[spark] class CarbonSQLDialect(context: HiveContext) extends HiveQLDialec
       // because hive can no parse carbon command
       case ce: MalformedCarbonCommandException =>
         throw ce
-      case _ => super.parse(sqlText)
+      case _ =>
+        HiveQl.parseSql(sqlText)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
index 76edc11..a989efe 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
@@ -22,6 +22,7 @@ import scala.collection.JavaConverters._
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.TableIdentifier._
 import org.apache.spark.sql.catalyst.expressions
 import org.apache.spark.sql.catalyst.expressions.{AttributeSet, _}
 import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, QueryPlanner}
@@ -134,7 +135,8 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
       val projectExprsNeedToDecode = new java.util.HashSet[Attribute]()
       val scan = CarbonScan(projectList.map(_.toAttribute),
         relation.carbonRelation,
-        predicates)(sqlContext)
+        predicates,
+        useUnsafeCoversion = false)(sqlContext)
       projectExprsNeedToDecode.addAll(scan.attributesNeedToDecode)
       if (projectExprsNeedToDecode.size() > 0) {
         val decoder = getCarbonDecoder(logicalRelation,
@@ -194,7 +196,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         ExecutedCommand(ShowAllTablesDetail(schemaName, plan.output)) :: Nil
       case DropTable(tableName, ifNotExists)
         if CarbonEnv.getInstance(sqlContext).carbonCatalog
-            .tableExists(TableIdentifier(tableName.toLowerCase))(sqlContext) =>
+            .tableExists(TableIdentifier(tableName.toLowerCase, None))(sqlContext) =>
         ExecutedCommand(DropTableCommand(ifNotExists, None, tableName.toLowerCase)) :: Nil
       case ShowAggregateTablesCommand(schemaName) =>
         ExecutedCommand(ShowAggregateTables(schemaName, plan.output)) :: Nil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
index 73cb3d5..fc74291 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
@@ -25,7 +25,7 @@ import scala.collection.mutable.ArrayBuffer
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.CatalystConf
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
+import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _}
 import org.apache.spark.sql.catalyst.rules.Rule

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
index 5811d3a..f9af61d 100644
--- a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
@@ -23,7 +23,6 @@ import scala.collection.JavaConversions._
 
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.execution.columnar.InMemoryRelation
 import org.apache.spark.sql.{DataFrame, Row, SQLContext}
 
 class QueryTest extends PlanTest {
@@ -70,21 +69,6 @@ class QueryTest extends PlanTest {
     }
   }
 
-  /**
-   * Asserts that a given [[DataFrame]] will be executed using the given number of cached results.
-   */
-  def assertCached(query: DataFrame, numCachedTables: Int = 1): Unit = {
-    val planWithCaching = query.queryExecution.withCachedData
-    val cachedData = planWithCaching collect {
-      case cached: InMemoryRelation => cached
-    }
-
-    assert(
-      cachedData.size == numCachedTables,
-      s"Expected query to contain $numCachedTables, but it actually had ${cachedData.size}\n" +
-        planWithCaching)
-  }
-
   protected def checkAnswer(df: DataFrame, expectedAnswer: Row): Unit = {
     checkAnswer(df, Seq(expectedAnswer))
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/19590dba/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 5df017a..d5f5e33 100644
--- a/pom.xml
+++ b/pom.xml
@@ -25,7 +25,7 @@
   <name>carbondata</name>
   <properties>
     <project.build.sourceEncoding>UTF-8</project.build.sourceEncoding>
-    <spark.version>1.6.1</spark.version>
+    <spark.version>1.5.2</spark.version>
     <scala.binary.version>2.10</scala.binary.version>
     <snappy.version>1.1.1.7</snappy.version>
     <hadoop.version>2.2.0</hadoop.version>
@@ -114,6 +114,19 @@
       </properties>
     </profile>
     <profile>
+      <id>spark-1.5.2</id>
+      <!-- default -->
+      <properties>
+        <spark.version>1.5.2</spark.version>
+      </properties>
+    </profile>
+    <profile>
+      <id>spark-1.6.1</id>
+      <properties>
+        <spark.version>1.6.1</spark.version>
+      </properties>
+    </profile>
+    <profile>
       <id>integration-test</id>
       <modules>
         <module>integration-testcases</module>


[51/56] [abbrv] incubator-carbondata git commit: Merge pull request #707 from ravipesala/merge3

Posted by jb...@apache.org.
Merge pull request #707 from ravipesala/merge3

Merging code from master to spark16 branch

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

Branch: refs/heads/master
Commit: 3c41deb8881831d1633d93e19a0e76de4ba7bcd2
Parents: bf7557d a76ca0f
Author: Liang Chen <ch...@huawei.com>
Authored: Tue Jun 21 14:01:10 2016 +0800
Committer: GitHub <no...@github.com>
Committed: Tue Jun 21 14:01:10 2016 +0800

----------------------------------------------------------------------
 .../core/carbon/path/CarbonTablePath.java       |  20 +-
 .../core/constants/CarbonCommonConstants.java   |  27 +
 .../fileperations/AtomicFileOperationsImpl.java |   3 +-
 .../store/filesystem/AbstractDFSCarbonFile.java | 217 +++++++
 .../store/filesystem/HDFSCarbonFile.java        | 249 ++------
 .../store/filesystem/LocalCarbonFile.java       |   1 +
 .../store/filesystem/ViewFSCarbonFile.java      | 126 ++++
 .../store/impl/DFSFileHolderImpl.java           | 183 ++++++
 .../datastorage/store/impl/FileFactory.java     |  36 +-
 .../store/impl/HDFSFileHolderImpl.java          | 186 ------
 .../TimeStampDirectDictionaryGenerator.java     |  17 +-
 .../core/locks/CarbonLockFactory.java           |  10 +-
 .../carbondata/core/locks/ZooKeeperLocking.java |  53 +-
 .../carbondata/core/locks/ZookeeperInit.java    |  64 ++
 .../org/carbondata/core/util/CarbonUtil.java    |  23 +-
 .../resolver/ConditionalFilterResolverImpl.java |   9 +-
 .../core/locks/ZooKeeperLockingTest.java        |   9 +-
 .../org/carbondata/examples/CarbonExample.scala |   8 +-
 .../examples/GenerateDictionaryExample.scala    |   2 +-
 .../AbstractDictionaryDecodedReadSupport.java   |   8 +-
 .../carbondata/hadoop/util/SchemaReader.java    |   3 +-
 .../hadoop/ft/CarbonInputMapperTest.java        |  12 +-
 .../hadoop/test/util/StoreCreator.java          |   4 +-
 integration-testcases/pom.xml                   |   2 +-
 .../sql/common/util/CarbonHiveContext.scala     |   4 +-
 .../allqueries/AllDataTypesTestCase2.scala      |   9 +-
 .../allqueries/AllDataTypesTestCase4.scala      |  72 +--
 .../allqueries/AllDataTypesTestCase5.scala      |   9 +-
 .../allqueries/AllDataTypesTestCase6.scala      |  27 +-
 integration/spark/pom.xml                       |   2 +-
 .../spark/merger/CarbonCompactionExecutor.java  |   1 +
 .../spark/merger/CarbonCompactionUtil.java      |  33 --
 .../spark/merger/CompactionCallable.java        |  68 +++
 .../spark/merger/RowResultMerger.java           |  17 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  20 +-
 .../spark/merger/CarbonDataMergerUtil.java      |  28 +-
 .../readsupport/SparkRowReadSupportImpl.java    |   7 +
 .../spark/sql/CarbonCatalystOperators.scala     |   2 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  47 +-
 .../execution/command/carbonTableSchema.scala   |  37 +-
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |  11 +
 .../scala/org/carbondata/spark/KeyVal.scala     |   9 +-
 .../spark/csv/CarbonCsvRelation.scala           | 242 ++++++++
 .../carbondata/spark/csv/DefaultSource.scala    |  94 +--
 .../spark/rdd/CarbonDataLoadRDD.scala           |  20 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        | 245 +++++---
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  27 +-
 .../org/carbondata/spark/rdd/Compactor.scala    |  15 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  53 +-
 .../src/test/resources/InvalidCsvFormatdata.csv |   3 -
 .../spark/src/test/resources/avgTest.csv        |  30 +-
 .../test/resources/compaction/compaction1.csv   |  10 +-
 .../test/resources/compaction/compaction2.csv   |  10 +-
 .../test/resources/compaction/compaction3.csv   |  10 +-
 integration/spark/src/test/resources/data2.csv  |   4 +-
 .../spark/src/test/resources/datasamplenull.csv |   3 +
 .../TestLoadDataWithEmptyArrayColumns.scala     |  11 +-
 .../AllDataTypesTestCaseAggregate.scala         |   4 +
 .../aggquery/AverageQueryTestCase.scala         |   7 +-
 .../AllDataTypesTestCaseAggregate.scala         |  10 +-
 .../testsuite/bigdecimal/TestBigDecimal.scala   |   7 +
 .../spark/testsuite/bigdecimal/TestBigInt.scala |   7 +
 .../DataCompactionCardinalityBoundryTest.scala  |  10 +-
 .../DataCompactionNoDictionaryTest.scala        |  29 +-
 .../datacompaction/DataCompactionTest.scala     |  12 +-
 .../dataload/DefaultSourceTestCase.scala        |  25 +-
 .../TestDataWithDicExcludeAndInclude.scala      |   9 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |   8 +-
 .../dataload/TestLoadTblNameIsKeyword.scala     |  96 +++
 .../dataretention/DataRetentionTestCase.scala   |  11 +-
 .../TimestampDataTypeNullDataTest.scala         |  92 +++
 .../filterexpr/FilterProcessorTestCase.scala    |   9 +
 .../NullMeasureValueTestCaseAggregate.scala     |  14 +-
 .../csvreaderstep/BlockDataHandler.java         | 579 +++++++++----------
 .../processing/csvreaderstep/CsvInput.java      |  10 +-
 .../processing/csvreaderstep/CsvInputMeta.java  |  66 +--
 .../graphgenerator/GraphGenerator.java          |  14 +-
 .../processing/mdkeygen/MDKeyGenStep.java       |   2 +-
 .../sortandgroupby/sortdata/SortDataRows.java   |   2 +-
 .../store/CarbonFactDataHandlerColumnar.java    |  30 +-
 .../store/CarbonFactDataHandlerModel.java       |  22 +
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |  29 +-
 .../FileStoreSurrogateKeyGenForCSV.java         |   2 +-
 .../util/CarbonDataProcessorUtil.java           |  48 +-
 84 files changed, 2302 insertions(+), 1304 deletions(-)
----------------------------------------------------------------------



[06/56] [abbrv] incubator-carbondata git commit: [Issue 618]Supported Spark 1.6 in Carbondata (#670)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DummyDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DummyDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DummyDoubleAggregator.java
deleted file mode 100644
index fa5a719..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DummyDoubleAggregator.java
+++ /dev/null
@@ -1,64 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DummyDoubleAggregator extends AbstractMeasureAggregatorDummy {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * aggregate value
-   */
-  private double aggVal;
-
-  @Override public void agg(double newVal) {
-    aggVal = newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(Object newVal) {
-    aggVal = (Double) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  @Override public Double getDoubleValue() {
-    return aggVal;
-  }
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Double) newValue;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DummyDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DummyLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DummyLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DummyLongAggregator.java
deleted file mode 100644
index 3663ec9..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DummyLongAggregator.java
+++ /dev/null
@@ -1,59 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DummyLongAggregator extends AbstractMeasureAggregatorDummy {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * aggregate value
-   */
-  private long aggVal;
-
-  @Override public void agg(Object newVal) {
-    aggVal = (Long) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  @Override public Long getLongValue() {
-    return aggVal;
-  }
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Long) newValue;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DummyLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/MaxAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/MaxAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/MaxAggregator.java
deleted file mode 100644
index 13094ca..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/MaxAggregator.java
+++ /dev/null
@@ -1,68 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description :
- * It will return max of values
- */
-public class MaxAggregator extends AbstractMaxAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -5850218739083899419L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the max aggregate value if aggregator
-   * passed as an argument will have value greater than aggVal
-   *
-   * @param aggregator MaxAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(maxAggregator.aggVal);
-      firstTime = false;
-    }
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MaxAggregator aggregator = new MaxAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MaxAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/MaxBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/MaxBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/MaxBigDecimalAggregator.java
deleted file mode 100644
index 3f754a9..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/MaxBigDecimalAggregator.java
+++ /dev/null
@@ -1,67 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description :
- * It will return max of values
- */
-public class MaxBigDecimalAggregator extends AbstractMaxAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -5850218739083899419L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the max aggregate value if aggregator
-   * passed as an argument will have value greater than aggVal
-   *
-   * @param aggregator MaxAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(maxAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MaxAggregator aggregator = new MaxAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MaxBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/MaxLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/MaxLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/MaxLongAggregator.java
deleted file mode 100644
index d3a4a74..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/MaxLongAggregator.java
+++ /dev/null
@@ -1,67 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description :
- * It will return max of values
- */
-public class MaxLongAggregator extends AbstractMaxAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -5850218739083899419L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the max aggregate value if aggregator
-   * passed as an argument will have value greater than aggVal
-   *
-   * @param aggregator MaxAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(maxAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MaxAggregator aggregator = new MaxAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MaxLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/MinAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/MinAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/MinAggregator.java
deleted file mode 100644
index 251b0f6..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/MinAggregator.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return min of values
- */
-public class MinAggregator extends AbstractMinAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8077547753784906280L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the min aggregate value if aggregator
-   * passed as an argument will have value less than aggVal
-   *
-   * @param aggregator MinAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MinAggregator minAggregator = (MinAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(minAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MinAggregator aggregator = new MinAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MinAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/MinBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/MinBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/MinBigDecimalAggregator.java
deleted file mode 100644
index 6f7db06..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/MinBigDecimalAggregator.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return min of values
- */
-public class MinBigDecimalAggregator extends AbstractMinAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8077547753784906280L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the min aggregate value if aggregator
-   * passed as an argument will have value less than aggVal
-   *
-   * @param aggregator MinAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MinAggregator minAggregator = (MinAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(minAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MinAggregator aggregator = new MinAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MinBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/MinLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/MinLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/MinLongAggregator.java
deleted file mode 100644
index 7cb6a54..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/MinLongAggregator.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.aggregator.impl;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return min of values
- */
-public class MinLongAggregator extends AbstractMinAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8077547753784906280L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the min aggregate value if aggregator
-   * passed as an argument will have value less than aggVal
-   *
-   * @param aggregator MinAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MinAggregator minAggregator = (MinAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(minAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MinAggregator aggregator = new MinAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MinLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/SumBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/SumBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/SumBigDecimalAggregator.java
deleted file mode 100644
index 36ecea3..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/SumBigDecimalAggregator.java
+++ /dev/null
@@ -1,178 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class SumBigDecimalAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 623750056131364540L;
-
-  /**
-   * aggregate value
-   */
-  private BigDecimal aggVal;
-
-  public SumBigDecimalAggregator() {
-    aggVal = new BigDecimal(0);
-    firstTime = false;
-  }
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (firstTime) {
-      aggVal = (BigDecimal) newVal;
-      firstTime = false;
-    } else {
-      aggVal = aggVal.add((BigDecimal) newVal);
-    }
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      BigDecimal value = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-      aggVal = aggVal.add(value);
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    byte[] bytes = DataTypeUtil.bigDecimalToByte(aggVal);
-    ByteBuffer allocate = ByteBuffer.allocate(4 + bytes.length);
-
-    allocate.putInt(bytes.length);
-    allocate.put(bytes);
-    allocate.rewind();
-    return allocate.array();
-  }
-
-  /**
-   * This method will return aggVal
-   *
-   * @return sum value
-   */
-  @Override public BigDecimal getBigDecimalValue() {
-    return aggVal;
-  }
-
-  /* Merge the value, it will update the sum aggregate value it will add new
-   * value to aggVal
-   *
-   * @param aggregator
-   *            SumAggregator
-   *
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    if (!aggregator.isFirstTime()) {
-      agg(aggregator.getBigDecimalValue());
-    }
-  }
-
-  /**
-   * This method return the sum value as an object
-   *
-   * @return sum value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (BigDecimal) newValue;
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = new BigDecimal(inPut.readUTF());
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeUTF(aggVal.toString());
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumBigDecimalAggregator aggr = new SumBigDecimalAggregator();
-    aggr.aggVal = aggVal;
-    aggr.firstTime = firstTime;
-    return aggr;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    byte[] valueByte = new byte[buffer.getInt()];
-    buffer.get(valueByte);
-    BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-    aggVal = aggVal.add(valueBigDecimal);
-    firstTime = false;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    BigDecimal value = getBigDecimalValue();
-    BigDecimal otherVal = o.getBigDecimalValue();
-    return value.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumBigDecimalAggregator)) {
-      return false;
-    }
-    SumBigDecimalAggregator o = (SumBigDecimalAggregator) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new SumBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctBigDecimalAggregator.java
deleted file mode 100644
index 33b9bd9..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctBigDecimalAggregator.java
+++ /dev/null
@@ -1,232 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * The sum distinct aggregator
- * Ex:
- * ID NAME Sales
- * 1 a 200
- * 2 a 100
- * 3 a 200
- * select sum(distinct sales) # would result 300
- */
-public class SumDistinctBigDecimalAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 6313463368629960155L;
-
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten the
-   * Aggregators. There is no aggregation expected after setting this value.
-   */
-  private BigDecimal computedFixedValue;
-
-  /**
-   *
-   */
-  private Set<BigDecimal> valueSet;
-
-  public SumDistinctBigDecimalAggregator() {
-    valueSet = new HashSet<BigDecimal>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * Distinct Aggregate function which update the Distinct set
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSet.add(
-        newVal instanceof BigDecimal ? (BigDecimal) newVal : new BigDecimal(newVal.toString()));
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    Iterator<BigDecimal> iterator = valueSet.iterator();
-    ByteBuffer buffer =
-        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    while (iterator.hasNext()) {
-      byte[] bytes = DataTypeUtil.bigDecimalToByte(iterator.next());
-      buffer.putInt(bytes.length);
-      buffer.put(bytes);
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(Set<BigDecimal> set2) {
-    valueSet.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    SumDistinctBigDecimalAggregator distinctAggregator =
-        (SumDistinctBigDecimalAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(distinctAggregator.valueSet);
-      firstTime = false;
-    }
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    if (computedFixedValue == null) {
-      BigDecimal result = new BigDecimal(0);
-      for (BigDecimal aValue : valueSet) {
-        result = result.add(aValue);
-      }
-      return result;
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Object getValueObject() {
-    return getBigDecimalValue();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (BigDecimal) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void writeData(DataOutput dataOutput) throws IOException {
-    if (computedFixedValue != null) {
-      byte[] bytes = DataTypeUtil.bigDecimalToByte(computedFixedValue);
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 4 + bytes.length);
-      byteBuffer.putInt(-1);
-      byteBuffer.putInt(bytes.length);
-      byteBuffer.put(bytes);
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    } else {
-      int length = valueSet.size() * 8 + valueSet.size() * 4;
-      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
-      byteBuffer.putInt(length);
-      for (BigDecimal val : valueSet) {
-        byte[] bytes =
-            val.toString().getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-        byteBuffer.putInt(-1);
-        byteBuffer.putInt(bytes.length);
-        byteBuffer.put(bytes);
-      }
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-
-    if (length == -1) {
-      computedFixedValue = new BigDecimal(inPut.readUTF());
-      valueSet = null;
-    } else {
-      length = length / 8;
-      valueSet = new HashSet<BigDecimal>(length + 1, 1.0f);
-      for (int i = 0; i < length; i++) {
-        valueSet.add(new BigDecimal(inPut.readUTF()));
-      }
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      byte[] valueByte = new byte[buffer.getInt()];
-      buffer.get(valueByte);
-      BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-      agg(valueBigDecimal);
-    }
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      return valueSet.size() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDistinctBigDecimalAggregator aggregator = new SumDistinctBigDecimalAggregator();
-    aggregator.valueSet = new HashSet<BigDecimal>(valueSet);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator msr) {
-    BigDecimal msrValObj = getBigDecimalValue();
-    BigDecimal otherVal = msr.getBigDecimalValue();
-
-    return msrValObj.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumDistinctBigDecimalAggregator)) {
-      return false;
-    }
-    SumDistinctBigDecimalAggregator o = (SumDistinctBigDecimalAggregator) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumDistinctBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctDoubleAggregator.java
deleted file mode 100644
index 6a40cf2..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctDoubleAggregator.java
+++ /dev/null
@@ -1,230 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * The sum distinct aggregator
- * Ex:
- * ID NAME Sales
- * 1 a 200
- * 2 a 100
- * 3 a 200
- * select sum(distinct sales) # would result 300
- */
-
-public class SumDistinctDoubleAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 6313463368629960155L;
-
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten the
-   * Aggregators. There is no aggregation expected after setting this value.
-   */
-  private Double computedFixedValue;
-
-  /**
-   *
-   */
-  private Set<Double> valueSet;
-
-  public SumDistinctDoubleAggregator() {
-    valueSet = new HashSet<Double>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-    valueSet.add(newVal);
-    firstTime = false;
-  }
-
-  /**
-   * Distinct Aggregate function which update the Distinct set
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSet.add(newVal instanceof Double ? (Double) newVal : new Double(newVal.toString()));
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    Iterator<Double> iterator = valueSet.iterator();
-    ByteBuffer buffer =
-        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    while (iterator.hasNext()) {
-      buffer.putDouble(iterator.next());
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(Set<Double> set2) {
-    valueSet.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    SumDistinctDoubleAggregator distinctAggregator = (SumDistinctDoubleAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(distinctAggregator.valueSet);
-      firstTime = false;
-    }
-  }
-
-  @Override public Double getDoubleValue() {
-    if (computedFixedValue == null) {
-      double result = 0;
-      for (Double aValue : valueSet) {
-        result += aValue;
-      }
-      return result;
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Object getValueObject() {
-    return getDoubleValue();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (Double) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void writeData(DataOutput dataOutput) throws IOException {
-    if (computedFixedValue != null) {
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
-      byteBuffer.putInt(-1);
-      byteBuffer.putDouble(computedFixedValue);
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    } else {
-      int length = valueSet.size() * 8;
-      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
-      byteBuffer.putInt(length);
-      for (double val : valueSet) {
-        byteBuffer.putDouble(val);
-      }
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-
-    if (length == -1) {
-      computedFixedValue = inPut.readDouble();
-      valueSet = null;
-    } else {
-      length = length / 8;
-      valueSet = new HashSet<Double>(length + 1, 1.0f);
-      for (int i = 0; i < length; i++) {
-        valueSet.add(inPut.readDouble());
-      }
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      agg(buffer.getDouble());
-    }
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      return valueSet.size() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDistinctDoubleAggregator aggregator = new SumDistinctDoubleAggregator();
-    aggregator.valueSet = new HashSet<Double>(valueSet);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator msr) {
-    double msrValObj = getDoubleValue();
-    double otherVal = msr.getDoubleValue();
-    if (msrValObj > otherVal) {
-      return 1;
-    }
-    if (msrValObj < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumDistinctDoubleAggregator)) {
-      return false;
-    }
-    SumDistinctDoubleAggregator o = (SumDistinctDoubleAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumDistinctDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctLongAggregator.java
deleted file mode 100644
index db84818..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDistinctLongAggregator.java
+++ /dev/null
@@ -1,220 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * The sum distinct aggregator
- * Ex:
- * ID NAME Sales
- * 1 a 200
- * 2 a 100
- * 3 a 200
- * select sum(distinct sales) # would result 300
- */
-
-public class SumDistinctLongAggregator extends AbstractMeasureAggregatorBasic {
-
-  private static final long serialVersionUID = 6313463368629960155L;
-
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten the
-   * Aggregators. There is no aggregation expected after setting this value.
-   */
-  private Long computedFixedValue;
-
-  /**
-   *
-   */
-  private Set<Long> valueSet;
-
-  public SumDistinctLongAggregator() {
-    valueSet = new HashSet<Long>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * Distinct Aggregate function which update the Distinct set
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSet.add(newVal instanceof Long ? (Long) newVal : Long.valueOf(newVal.toString()));
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    Iterator<Long> iterator = valueSet.iterator();
-    ByteBuffer buffer =
-        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    while (iterator.hasNext()) {
-      buffer.putLong(iterator.next());
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(Set<Long> set2) {
-    valueSet.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    SumDistinctLongAggregator distinctAggregator = (SumDistinctLongAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(distinctAggregator.valueSet);
-      firstTime = false;
-    }
-  }
-
-  @Override public Long getLongValue() {
-    if (computedFixedValue == null) {
-      long result = 0;
-      for (Long aValue : valueSet) {
-        result += aValue;
-      }
-      return result;
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Object getValueObject() {
-    return getLongValue();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (Long) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void writeData(DataOutput dataOutput) throws IOException {
-    if (computedFixedValue != null) {
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
-      byteBuffer.putInt(-1);
-      byteBuffer.putLong(computedFixedValue);
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    } else {
-      int length = valueSet.size() * 8;
-      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
-      byteBuffer.putInt(length);
-      for (long val : valueSet) {
-        byteBuffer.putLong(val);
-      }
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-
-    if (length == -1) {
-      computedFixedValue = inPut.readLong();
-      valueSet = null;
-    } else {
-      length = length / 8;
-      valueSet = new HashSet<Long>(length + 1, 1.0f);
-      for (int i = 0; i < length; i++) {
-        valueSet.add(inPut.readLong());
-      }
-    }
-
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      agg(buffer.getLong());
-    }
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      return valueSet.size() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDistinctLongAggregator aggregator = new SumDistinctLongAggregator();
-    aggregator.valueSet = new HashSet<Long>(valueSet);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator msr) {
-    long msrValObj = getLongValue();
-    long otherVal = msr.getLongValue();
-    if (msrValObj > otherVal) {
-      return 1;
-    }
-    if (msrValObj < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumDistinctLongAggregator)) {
-      return false;
-    }
-    SumDistinctLongAggregator o = (SumDistinctLongAggregator) obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumDistinctLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/SumDoubleAggregator.java
deleted file mode 100644
index ee340f7..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/SumDoubleAggregator.java
+++ /dev/null
@@ -1,178 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class SumDoubleAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 623750056131364540L;
-
-  /**
-   * aggregate value
-   */
-  private double aggVal;
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(double newVal) {
-    aggVal += newVal;
-    firstTime = false;
-  }
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    aggVal += ((Number) newVal).doubleValue();
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal+= dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putDouble(aggVal);
-    return buffer.array();
-  }
-
-  /**
-   * This method will return aggVal
-   *
-   * @return sum value
-   */
-
-  @Override public Double getDoubleValue() {
-    return aggVal;
-  }
-
-  /* Merge the value, it will update the sum aggregate value it will add new
-   * value to aggVal
-   *
-   * @param aggregator  SumAggregator
-   *
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    if (!aggregator.isFirstTime()) {
-      agg(aggregator.getDoubleValue());
-    }
-  }
-
-  /**
-   * This method return the sum value as an object
-   *
-   * @return sum value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Double) newValue;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readDouble();
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeDouble(aggVal);
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDoubleAggregator aggr = new SumDoubleAggregator();
-    aggr.aggVal = aggVal;
-    aggr.firstTime = firstTime;
-    return aggr;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    aggVal += ByteBuffer.wrap(value).getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    double value = getDoubleValue();
-    double otherVal = o.getDoubleValue();
-    if (value > otherVal) {
-      return 1;
-    }
-    if (value < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof SumDoubleAggregator)) {
-      return false;
-    }
-    SumDoubleAggregator o = (SumDoubleAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new SumDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/SumLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/SumLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/SumLongAggregator.java
deleted file mode 100644
index 193b2b2..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/SumLongAggregator.java
+++ /dev/null
@@ -1,164 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class SumLongAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 623750056131364540L;
-
-  /**
-   * aggregate value
-   */
-  private long aggVal;
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    aggVal += (long) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal+= dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.LONG_SIZE_IN_BYTE);
-    buffer.putLong(aggVal);
-    return buffer.array();
-  }
-
-  /**
-   * This method will return aggVal
-   *
-   * @return sum value
-   */
-  @Override public Long getLongValue() {
-    return aggVal;
-  }
-
-  /* Merge the value, it will update the sum aggregate value it will add new
-   * value to aggVal
-   *
-   * @param aggregator SumAggregator
-   *
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    if (!aggregator.isFirstTime()) {
-      agg(aggregator.getLongValue());
-    }
-  }
-
-  /**
-   * This method return the sum value as an object
-   *
-   * @return sum long value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (long) newValue;
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readLong();
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeLong(aggVal);
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumLongAggregator aggr = new SumLongAggregator();
-    aggr.aggVal = aggVal;
-    aggr.firstTime = firstTime;
-    return aggr;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    aggVal += ByteBuffer.wrap(value).getLong();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    Long value = getLongValue();
-    Long otherVal = o.getLongValue();
-    if (value > otherVal) {
-      return 1;
-    }
-    if (value < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof SumLongAggregator)) {
-      return false;
-    }
-    SumLongAggregator o = (SumLongAggregator) obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java
new file mode 100644
index 0000000..24eff08
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java
@@ -0,0 +1,28 @@
+/*
+ * 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.carbondata.query.aggregator.impl.avg;
+
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+public abstract class AbstractAvgAggregator extends AbstractMeasureAggregatorBasic {
+
+  public abstract Object[] getAvgState();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java
new file mode 100644
index 0000000..c6ba55d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java
@@ -0,0 +1,218 @@
+/*
+ * 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.carbondata.query.aggregator.impl.avg;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.DataTypeUtil;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class AvgBigDecimalAggregator extends AbstractAvgAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 5463736686281089871L;
+
+  /**
+   * total number of aggregate values
+   */
+  protected double count;
+
+  /**
+   * aggregate value
+   */
+  protected BigDecimal aggVal;
+
+  public AvgBigDecimalAggregator() {
+    aggVal = new BigDecimal(0);
+  }
+
+  /**
+   * Average Aggregate function which will add all the aggregate values and it
+   * will increment the total count every time, for average value
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    if (newVal instanceof byte[]) {
+      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
+      buffer.rewind();
+      while (buffer.hasRemaining()) {
+        byte[] valueByte = new byte[buffer.getInt()];
+        buffer.get(valueByte);
+        BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
+        aggVal = aggVal.add(valueBigDecimal);
+
+        count += buffer.getDouble();
+        firstTime = false;
+      }
+      return;
+    }
+
+    if (firstTime) {
+      aggVal = (BigDecimal) newVal;
+      firstTime = false;
+    } else {
+      aggVal = aggVal.add((BigDecimal) newVal);
+    }
+    count++;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      BigDecimal value = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+      aggVal = aggVal.add(value);
+      firstTime = false;
+      count++;
+    }
+  }
+
+  @Override public Object[] getAvgState() {
+    return new Object[]{aggVal, count};
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    if (firstTime) {
+      return new byte[0];
+    }
+    byte[] bytes = DataTypeUtil.bigDecimalToByte(aggVal);
+    ByteBuffer allocate =
+        ByteBuffer.allocate(4 + bytes.length + CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    allocate.putInt(bytes.length);
+    allocate.put(bytes);
+    allocate.putDouble(count);
+    allocate.rewind();
+
+    return allocate.array();
+  }
+
+  /**
+   * Return the average of the aggregate values
+   *
+   * @return average aggregate value
+   */
+  @Override public BigDecimal getBigDecimalValue() {
+    return aggVal.divide(new BigDecimal(count), 6);
+  }
+
+  /**
+   * This method merge the aggregated value, in average aggregator it will add
+   * count and aggregate value
+   *
+   * @param aggregator Avg Aggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    AvgBigDecimalAggregator avgAggregator = (AvgBigDecimalAggregator) aggregator;
+    if (!avgAggregator.isFirstTime()) {
+      aggVal = aggVal.add(avgAggregator.aggVal);
+      count += avgAggregator.count;
+      firstTime = false;
+    }
+  }
+
+  /**
+   * This method return the average value as an object
+   *
+   * @return average value as an object
+   */
+  @Override public Object getValueObject() {
+    return aggVal.divide(new BigDecimal(count));
+  }
+
+  /**
+   * @see MeasureAggregator#setNewValue(Object)
+   */
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (BigDecimal) newValue;
+    count = 1;
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+    output.writeBoolean(firstTime);
+    output.writeUTF(aggVal.toString());
+    output.writeDouble(count);
+
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    firstTime = inPut.readBoolean();
+    aggVal = new BigDecimal(inPut.readUTF());
+    count = inPut.readDouble();
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    AvgBigDecimalAggregator avg = new AvgBigDecimalAggregator();
+    avg.aggVal = aggVal;
+    avg.count = count;
+    avg.firstTime = firstTime;
+    return avg;
+  }
+
+  @Override public int compareTo(MeasureAggregator o) {
+    BigDecimal val = getBigDecimalValue();
+    BigDecimal otherVal = o.getBigDecimalValue();
+
+    return val.compareTo(otherVal);
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof AvgBigDecimalAggregator)) {
+      return false;
+    }
+    AvgBigDecimalAggregator o = (AvgBigDecimalAggregator) obj;
+    return getBigDecimalValue().equals(o.getBigDecimalValue());
+  }
+
+  @Override public int hashCode() {
+    return getBigDecimalValue().hashCode();
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+
+    byte[] valueByte = new byte[buffer.getInt()];
+    buffer.get(valueByte);
+    BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
+    aggVal = aggVal.add(valueBigDecimal);
+    count += buffer.getDouble();
+    firstTime = false;
+  }
+
+  public String toString() {
+    return (aggVal.divide(new BigDecimal(count))) + "";
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new AvgBigDecimalAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java
new file mode 100644
index 0000000..bacff29
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java
@@ -0,0 +1,210 @@
+/*
+ * 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.carbondata.query.aggregator.impl.avg;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class AvgDoubleAggregator extends AbstractAvgAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 5463736686281089871L;
+
+  /**
+   * total number of aggregate values
+   */
+  protected double count;
+
+  /**
+   * aggregate value
+   */
+  protected double aggVal;
+
+  /**
+   * Average Aggregate function which will add all the aggregate values and it
+   * will increment the total count every time, for average value
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(double newVal) {
+    aggVal += newVal;
+    count++;
+    firstTime = false;
+  }
+
+  /**
+   * Average Aggregate function which will add all the aggregate values and it
+   * will increment the total count every time, for average value
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    if (newVal instanceof byte[]) {
+      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
+      buffer.rewind();
+      while (buffer.hasRemaining()) {
+        aggVal += buffer.getDouble();
+        count += buffer.getDouble();
+        firstTime = false;
+      }
+      return;
+    }
+    aggVal += ((Number) newVal).doubleValue();
+    count++;
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal += dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+      count++;
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    if (firstTime) {
+      return new byte[0];
+    }
+    ByteBuffer buffer = ByteBuffer.allocate(2 * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    buffer.putDouble(aggVal);
+    buffer.putDouble(count);
+    return buffer.array();
+  }
+
+  /**
+   * Return the average of the aggregate values
+   *
+   * @return average aggregate value
+   */
+  @Override public Double getDoubleValue() {
+    return aggVal / count;
+  }
+
+  @Override public Object[] getAvgState() {
+    return new Object[]{aggVal, count};
+  }
+
+  /**
+   * This method merge the aggregated value, in average aggregator it will add
+   * count and aggregate value
+   *
+   * @param aggregator Avg Aggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    AvgDoubleAggregator avgAggregator = (AvgDoubleAggregator) aggregator;
+    if (!avgAggregator.isFirstTime()) {
+      aggVal += avgAggregator.aggVal;
+      count += avgAggregator.count;
+      firstTime = false;
+    }
+  }
+
+  /**
+   * This method return the average value as an object
+   *
+   * @return average value as an object
+   */
+  @Override public Object getValueObject() {
+    return aggVal / count;
+  }
+
+  /**
+   * @see MeasureAggregator#setNewValue(Object)
+   */
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (Double) newValue;
+    count = 1;
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+    output.writeBoolean(firstTime);
+    output.writeDouble(aggVal);
+    output.writeDouble(count);
+
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    firstTime = inPut.readBoolean();
+    aggVal = inPut.readDouble();
+    count = inPut.readDouble();
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    AvgDoubleAggregator avg = new AvgDoubleAggregator();
+    avg.aggVal = aggVal;
+    avg.count = count;
+    avg.firstTime = firstTime;
+    return avg;
+  }
+
+  @Override public int compareTo(MeasureAggregator o) {
+    double val = getDoubleValue();
+    double otherVal = o.getDoubleValue();
+    if (val > otherVal) {
+      return 1;
+    }
+    if (val < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof AvgDoubleAggregator)) {
+      return false;
+    }
+    AvgDoubleAggregator o = (AvgDoubleAggregator)obj;
+    return getDoubleValue().equals(o.getDoubleValue());
+  }
+
+  @Override public int hashCode() {
+    return getDoubleValue().hashCode();
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    aggVal += buffer.getDouble();
+    count += buffer.getDouble();
+    firstTime = false;
+  }
+
+  public String toString() {
+    return (aggVal / count) + "";
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new AvgDoubleAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java
new file mode 100644
index 0000000..6290de6
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java
@@ -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.carbondata.query.aggregator.impl.avg;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class AvgLongAggregator extends AbstractAvgAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 5463736686281089871L;
+
+  /**
+   * total number of aggregate values
+   */
+  protected double count;
+
+  /**
+   * aggregate value
+   */
+  protected long aggVal;
+
+  /**
+   * Average Aggregate function which will add all the aggregate values and it
+   * will increment the total count every time, for average value
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    if (newVal instanceof byte[]) {
+      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
+      buffer.rewind();
+      while (buffer.hasRemaining()) {
+        aggVal += buffer.getLong();
+        count += buffer.getDouble();
+        firstTime = false;
+      }
+      return;
+    }
+    aggVal += (Long) newVal;
+    count++;
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal += dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+      count++;
+      firstTime = false;
+    }
+  }
+
+  @Override public Object[] getAvgState() {
+    return new Object[]{aggVal, count};
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    if (firstTime) {
+      return new byte[0];
+    }
+    ByteBuffer buffer = ByteBuffer.allocate(
+        CarbonCommonConstants.LONG_SIZE_IN_BYTE + CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    buffer.putLong(aggVal);
+    buffer.putDouble(count);
+    return buffer.array();
+  }
+
+  @Override public Long getLongValue() {
+    return aggVal / (long) count;
+  }
+
+  /**
+   * This method merge the aggregated value, in average aggregator it will add
+   * count and aggregate value
+   *
+   * @param aggregator Avg Aggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    AvgLongAggregator avgAggregator = (AvgLongAggregator) aggregator;
+    if (!avgAggregator.isFirstTime()) {
+      aggVal += avgAggregator.aggVal;
+      count += avgAggregator.count;
+      firstTime = false;
+    }
+  }
+
+  /**
+   * This method return the average value as an object
+   *
+   * @return average value as an object
+   */
+  @Override public Object getValueObject() {
+    return aggVal / count;
+  }
+
+  /**
+   * @see MeasureAggregator#setNewValue(Object)
+   */
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (Long) newValue;
+    count = 1;
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+    output.writeBoolean(firstTime);
+    output.writeLong(aggVal);
+    output.writeDouble(count);
+
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    firstTime = inPut.readBoolean();
+    aggVal = inPut.readLong();
+    count = inPut.readDouble();
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    AvgLongAggregator avg = new AvgLongAggregator();
+    avg.aggVal = aggVal;
+    avg.count = count;
+    avg.firstTime = firstTime;
+    return avg;
+  }
+
+  @Override public int compareTo(MeasureAggregator o) {
+    long val = getLongValue();
+    long otherVal = o.getLongValue();
+    if (val > otherVal) {
+      return 1;
+    } else if (val < otherVal) {
+      return -1;
+    } else {
+      return 0;
+    }
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof AvgLongAggregator)) {
+      return false;
+    }
+    AvgLongAggregator o = (AvgLongAggregator)obj;
+    return getLongValue().equals(o.getLongValue());
+  }
+
+  @Override public int hashCode() {
+    return getLongValue().hashCode();
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    aggVal += buffer.getLong();
+    count += buffer.getDouble();
+    firstTime = false;
+  }
+
+  public String toString() {
+    return (aggVal / count) + "";
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new AvgLongAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java
new file mode 100644
index 0000000..352e7aa
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java
@@ -0,0 +1,204 @@
+/*
+ * 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.carbondata.query.aggregator.impl.count;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description : It will return total count of values
+ */
+public class CountAggregator implements MeasureAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 2678878935295306313L;
+
+  /**
+   * aggregate value
+   */
+  private double aggVal;
+
+  /**
+   * Count Aggregate function which update the total count
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(double newVal) {
+    aggVal++;
+  }
+
+  /**
+   * Count Aggregate function which update the total count
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    aggVal++;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal++;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    buffer.putDouble(aggVal);
+    return buffer.array();
+  }
+
+  /**
+   * Returns the total count
+   *
+   * @return total count
+   */
+  @Override public Double getDoubleValue() {
+    return aggVal;
+  }
+
+  @Override public Long getLongValue() {
+    return (long) aggVal;
+  }
+
+  @Override public BigDecimal getBigDecimalValue() {
+    return new BigDecimal(aggVal);
+  }
+
+  /**
+   * Merge the total count with the aggregator
+   *
+   * @param aggregator count aggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    CountAggregator countAggregator = (CountAggregator) aggregator;
+    aggVal += countAggregator.aggVal;
+  }
+
+  /**
+   * Overloaded Aggregate function will be used for Aggregate tables because
+   * aggregate table will have fact_count as a measure. It will update the
+   * total count
+   *
+   * @param newVal
+   *            new value
+   * @param factCount
+   *            total fact count
+   *
+   */
+  //    @Override
+  //    public void agg(double newVal, double factCount)
+  //    {
+  //        agg(newVal, null, 0, 0);
+  //    }
+
+  /**
+   * This method return the count value as an object
+   *
+   * @return count value as an object
+   */
+
+  @Override public Object getValueObject() {
+    return aggVal;
+  }
+
+  /**
+   * @see MeasureAggregator#setNewValue(Object)
+   */
+  @Override public void setNewValue(Object newValue) {
+    aggVal += Double.parseDouble(String.valueOf(newValue));
+  }
+
+  @Override public boolean isFirstTime() {
+    return false;
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+    output.writeDouble(aggVal);
+
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    aggVal = inPut.readDouble();
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    CountAggregator aggregator = new CountAggregator();
+    aggregator.aggVal = aggVal;
+    return aggregator;
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    aggVal += buffer.getDouble();
+  }
+
+  @Override public int compareTo(MeasureAggregator obj) {
+    double val = getDoubleValue();
+    double otherVal = obj.getDoubleValue();
+    if (val > otherVal) {
+      return 1;
+    }
+    if (val < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof CountAggregator)) {
+      return false;
+    }
+    CountAggregator o = (CountAggregator)obj;
+    return getDoubleValue().equals(o.getDoubleValue());
+  }
+
+  @Override public int hashCode() {
+    return getDoubleValue().hashCode();
+  }
+
+  @Override public MeasureAggregator get() {
+    return this;
+  }
+
+  public String toString() {
+    return aggVal + "";
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new CountAggregator();
+  }
+
+}


[49/56] [abbrv] incubator-carbondata git commit: Deleting unnecessary code added after merge from master

Posted by jb...@apache.org.
Deleting unnecessary code added after merge from master


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

Branch: refs/heads/master
Commit: 9d89d69e77bb63bef017b29f599282bd2ad9d2fd
Parents: 7972709
Author: ravipesala <ra...@gmail.com>
Authored: Mon Jun 20 21:46:47 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Mon Jun 20 21:46:47 2016 +0530

----------------------------------------------------------------------
 .../aggregator/impl/AvgTimestampAggregator.java | 113 ------------------
 .../aggregator/impl/SumTimestampAggregator.java |  89 --------------
 .../DirectDictionaryDimensionAggregator.java    | 117 -------------------
 3 files changed, 319 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d89d69e/core/src/main/java/org/carbondata/query/aggregator/impl/AvgTimestampAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgTimestampAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AvgTimestampAggregator.java
deleted file mode 100644
index 9058d64..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgTimestampAggregator.java
+++ /dev/null
@@ -1,113 +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.carbondata.query.aggregator.impl;
-
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class AvgTimestampAggregator extends AvgDoubleAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 5463736686281089871L;
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        aggVal += buffer.getDouble();
-        count += buffer.getDouble();
-        firstTime = false;
-      }
-    } else if (newVal instanceof Timestamp) {
-      aggVal += ((Timestamp) newVal).getTime();
-      count++;
-      firstTime = false;
-    } else if (newVal instanceof Number) {
-      aggVal += ((Number) newVal).doubleValue();
-      count++;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Return the average of the aggregate values
-   *
-   * @return average aggregate value
-   */
-  @Override public Double getDoubleValue() {
-    return aggVal / (count * 1000L);
-  }
-
-  /**
-   * This method return the average value as an object
-   *
-   * @return average value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal / (count * 1000L);
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Double) newValue;
-    count = 1;
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    AvgTimestampAggregator avg = new AvgTimestampAggregator();
-    avg.aggVal = aggVal;
-    avg.count = count;
-    avg.firstTime = firstTime;
-    return avg;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof AvgTimestampAggregator)) {
-      return false;
-    }
-    AvgTimestampAggregator o = (AvgTimestampAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  public String toString() {
-    return (aggVal / (count * 1000L)) + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new AvgTimestampAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d89d69e/core/src/main/java/org/carbondata/query/aggregator/impl/SumTimestampAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/SumTimestampAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/SumTimestampAggregator.java
deleted file mode 100644
index fd8a201..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/SumTimestampAggregator.java
+++ /dev/null
@@ -1,89 +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.carbondata.query.aggregator.impl;
-
-import java.sql.Timestamp;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class SumTimestampAggregator extends SumDoubleAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 623750056131364540L;
-
-  /**
-   * aggregate value
-   */
-  private double aggVal;
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof Timestamp) {
-      aggVal += ((Timestamp) newVal).getTime();
-      firstTime = false;
-    } else if (newVal instanceof Number) {
-      aggVal += ((Number) newVal).doubleValue();
-      firstTime = false;
-    }
-  }
-
-  /**
-   * This method will return aggVal
-   *
-   * @return sum value
-   */
-
-  @Override public Double getDoubleValue() {
-    return aggVal / 1000L;
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumTimestampAggregator aggr = new SumTimestampAggregator();
-    aggr.aggVal = aggVal;
-    aggr.firstTime = firstTime;
-    return aggr;
-  }
-
-  public String toString() {
-    return (aggVal / 1000L) + "";
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumTimestampAggregator)) {
-      return false;
-    }
-    SumTimestampAggregator o = (SumTimestampAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new SumTimestampAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d89d69e/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
deleted file mode 100644
index 8794021..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.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.carbondata.query.carbon.aggregator.dimension.impl;
-
-import java.nio.ByteBuffer;
-import java.sql.Timestamp;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Class which will be used to aggregate the direct dictionary dimension data
- */
-public class DirectDictionaryDimensionAggregator implements DimensionDataAggregator {
-
-  /**
-   * info object which store information about dimension to be aggregated
-   */
-  private DimensionAggregatorInfo dimensionAggeragtorInfo;
-
-  /**
-   * start index of the aggregator for current dimension column
-   */
-  private int aggregatorStartIndex;
-
-  /**
-   * buffer used to convert mdkey to surrogate key
-   */
-  private ByteBuffer buffer;
-
-  /**
-   * data index in the file
-   */
-  private int blockIndex;
-
-  /**
-   * to store index which will be used to aggregate
-   * number type value like sum avg
-   */
-  private int[] numberTypeAggregatorIndex;
-
-  /**
-   * DirectDictionaryGenerator
-   */
-  private DirectDictionaryGenerator directDictionaryGenerator;
-
-  /**
-   * to store index which will be used to aggregate
-   * actual type value like max, min, dictinct count
-   */
-  private int[] actualTypeAggregatorIndex;
-
-  public DirectDictionaryDimensionAggregator(DimensionAggregatorInfo dimensionAggeragtorInfo,
-      int aggregatorStartIndex, int blockIndex) {
-    this.dimensionAggeragtorInfo = dimensionAggeragtorInfo;
-    this.aggregatorStartIndex = aggregatorStartIndex;
-    this.blockIndex = blockIndex;
-    buffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
-    numberTypeAggregatorIndex =
-        QueryUtil.getNumberTypeIndex(this.dimensionAggeragtorInfo.getAggList());
-    actualTypeAggregatorIndex =
-        QueryUtil.getActualTypeIndex(this.dimensionAggeragtorInfo.getAggList());
-    directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-        .getDirectDictionaryGenerator(this.dimensionAggeragtorInfo.getDim().getDataType());
-  }
-
-  /**
-   * Below method will be used to aggregate the dimension data
-   *
-   * @param scannedResult scanned result
-   * @param aggeragtor    aggregator used to aggregate the data
-   */
-  @Override public void aggregateDimensionData(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggeragtor) {
-    byte[] dimensionData = scannedResult.getDimensionKey(blockIndex);
-    int surrogateKey = CarbonUtil.getSurrogateKey(dimensionData, buffer);
-    Object valueFromSurrogate = directDictionaryGenerator.getValueFromSurrogate(surrogateKey);
-    if (null != valueFromSurrogate) {
-      Timestamp dataBasedOnDataType = new Timestamp((long) valueFromSurrogate / 1000);
-
-      if (actualTypeAggregatorIndex.length > 0) {
-        for (int j = 0; j < actualTypeAggregatorIndex.length; j++) {
-          aggeragtor[aggregatorStartIndex + actualTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-        }
-      }
-      if (numberTypeAggregatorIndex.length > 0) {
-        for (int j = 0; j < numberTypeAggregatorIndex.length; j++) {
-          aggeragtor[aggregatorStartIndex + numberTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-        }
-      }
-    }
-  }
-
-}


[13/56] [abbrv] incubator-carbondata git commit: [BUG] Zookeeper lock changes to allow exclusive locks for different tables (#688)

Posted by jb...@apache.org.
[BUG] Zookeeper lock changes to allow exclusive locks for different tables (#688)

added schema name and table name to Zookeeper lock location

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

Branch: refs/heads/master
Commit: f479e7efecfb10c16ac994a4c933d59efa6bb685
Parents: 6c2469d
Author: ravikiran23 <ra...@gmail.com>
Authored: Fri Jun 17 23:12:33 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Fri Jun 17 23:12:33 2016 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  5 ++++
 .../core/locks/CarbonLockFactory.java           |  2 +-
 .../carbondata/core/locks/ZooKeeperLocking.java | 26 +++++++++++++++++---
 .../core/locks/ZooKeeperLockingTest.java        |  7 ++++--
 4 files changed, 33 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f479e7ef/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
index bf0fd0b..6618ac0 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -1029,6 +1029,11 @@ public final class CarbonCommonConstants {
   public static final String CARBON_LOCK_TYPE_HDFS =
       "HDFSLOCK";
 
+  /**
+   * Lock file in zoo keeper will be of this name.
+   */
+  public static final String ZOOKEEPER_LOCK = "zookeeperLock";
+
   private CarbonCommonConstants() {
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f479e7ef/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java b/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java
index 232c2b8..99f4ab1 100644
--- a/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java
+++ b/core/src/main/java/org/carbondata/core/locks/CarbonLockFactory.java
@@ -49,7 +49,7 @@ public class CarbonLockFactory {
         return new LocalFileLock(location, lockUsage);
 
       case CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER:
-        return new ZooKeeperLocking(lockUsage);
+        return new ZooKeeperLocking(location, lockUsage);
 
       case CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS:
         return new HdfsFileLock(location, lockUsage);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f479e7ef/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java b/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java
index f7f5d0c..8b04ac1 100644
--- a/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java
+++ b/core/src/main/java/org/carbondata/core/locks/ZooKeeperLocking.java
@@ -50,6 +50,11 @@ public class ZooKeeperLocking extends AbstractCarbonLock {
   private static final String zooKeeperLocation = CarbonCommonConstants.ZOOKEEPER_LOCATION;
 
   /**
+   * Unique folder for each cube with SchemaName_CubeName
+   */
+  private final String tableIdFolder;
+
+  /**
    * lockName is the name of the lock to use. This name should be same for every process that want
    * to share the same lock
    */
@@ -65,17 +70,30 @@ public class ZooKeeperLocking extends AbstractCarbonLock {
   /**
    * @param lockUsage
    */
-  public ZooKeeperLocking(LockUsage lockUsage) {
-    this.lockName = CarbonCommonConstants.METADATA_LOCK;
+  public ZooKeeperLocking(String location, LockUsage lockUsage) {
+    this.lockName = CarbonCommonConstants.ZOOKEEPER_LOCK;
     this.lockTypeFolder = zooKeeperLocation;
+    location = location.replace("\\", "/");
+    String tempStr = location.substring(0, location.lastIndexOf('/'));
+    String schemaName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
+
+    String cubeName = location.substring(location.lastIndexOf('/') + 1, location.length());
+
+    this.tableIdFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + schemaName
+        + '.' + cubeName;
 
     zk = ZookeeperInit.getInstance().getZookeeper();
 
-    this.lockTypeFolder =
-        zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + lockUsage.toString();
+    this.lockTypeFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + schemaName
+        + '.' + cubeName + CarbonCommonConstants.FILE_SEPARATOR
+        + lockUsage.toString();
     try {
       createBaseNode();
       // if exists returns null then path doesnt exist. so creating.
+      if (null == zk.exists(this.tableIdFolder, true)) {
+        zk.create(this.tableIdFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+      }
+      // if exists returns null then path doesnt exist. so creating.
       if (null == zk.exists(this.lockTypeFolder, true)) {
         zk.create(this.lockTypeFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f479e7ef/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java b/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
index 5b21dde..3bf3993 100644
--- a/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
+++ b/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
@@ -78,10 +78,13 @@ public class ZooKeeperLockingTest {
 
     ZookeeperInit zki = ZookeeperInit.getInstance("127.0.0.1:" + freePort);
 
-    ZooKeeperLocking zkl = new ZooKeeperLocking(LockUsage.METADATA_LOCK);
+    ZooKeeperLocking zkl =
+        new ZooKeeperLocking("D:/carbondata/examples/target/store/default/t3/Metadata",
+            LockUsage.METADATA_LOCK);
     Assert.assertTrue(zkl.lock());
 
-    ZooKeeperLocking zk2 = new ZooKeeperLocking(LockUsage.METADATA_LOCK);
+    ZooKeeperLocking zk2 = new ZooKeeperLocking(
+        "D:/carbondata/examples/target/store/default/t3/Metadata", LockUsage.METADATA_LOCK);
     Assert.assertTrue(!zk2.lock());
 
     Assert.assertTrue(zkl.unlock());


[14/56] [abbrv] incubator-carbondata git commit: [Issue - 685] Support null or empty value for timestamp column (#686)

Posted by jb...@apache.org.
[Issue - 685] Support null or empty value for timestamp column (#686)



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

Branch: refs/heads/master
Commit: 91951acf2f89a0f5d0cf340a273f5ede8a08afb2
Parents: f479e7e
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Fri Jun 17 23:43:25 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Fri Jun 17 23:43:25 2016 +0530

----------------------------------------------------------------------
 .../TimeStampDirectDictionaryGenerator.java     | 21 +++--
 .../DirectDictionaryDimensionAggregator.java    | 21 +++--
 .../resolver/ConditionalFilterResolverImpl.java |  9 +-
 .../spark/src/test/resources/datasamplenull.csv |  3 +
 .../TimestampDataTypeNullDataTest.scala         | 92 ++++++++++++++++++++
 5 files changed, 127 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/91951acf/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
index d37dcf1..060adcb 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
@@ -1,4 +1,3 @@
-
 /*
  * Licensed to the Apache Software Foundation (ASF) under one
  * or more contributor license agreements.  See the NOTICE file
@@ -115,22 +114,27 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
         .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
             CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
     timeParser.setLenient(false);
+    if (null == memberStr || memberStr.trim().isEmpty() || memberStr
+        .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+      return 1;
+    }
     Date dateToStr = null;
     try {
       dateToStr = timeParser.parse(memberStr);
     } catch (ParseException e) {
-      LOGGER.error("Cannot convert" + TIMESTAMP.toString() + " to Time/Long type value"
-          + e.getMessage());
+      LOGGER.error(
+          "Cannot convert" + TIMESTAMP.toString() + " to Time/Long type value" + e.getMessage());
     }
+    //adding +2 to reserve the first cuttOffDiff value for null or empty date
     if (null == dateToStr) {
       return -1;
     } else {
       if (cutOffTimeStamp >= 0) {
         int keyValue = (int) ((dateToStr.getTime() - cutOffTimeStamp) / granularityFactor);
-        return keyValue < 0 ? -1 : keyValue;
+        return keyValue < 0 ? -1 : keyValue + 2;
       } else {
         int keyValue = (int) (dateToStr.getTime() / granularityFactor);
-        return keyValue < 0 ? -1 : keyValue;
+        return keyValue < 0 ? -1 : keyValue + 2;
       }
     }
   }
@@ -142,11 +146,14 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
    * @return member value/actual value Date
    */
   @Override public Object getValueFromSurrogate(int key) {
+    if (key == 1) {
+      return null;
+    }
     long timeStamp = 0;
     if (cutOffTimeStamp >= 0) {
-      timeStamp = (key * granularityFactor + cutOffTimeStamp);
+      timeStamp = ((key - 2) * granularityFactor + cutOffTimeStamp);
     } else {
-      timeStamp = key * granularityFactor;
+      timeStamp = (key - 2) * granularityFactor;
     }
     return timeStamp * 1000L;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/91951acf/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
index 971e4cc..8794021 100644
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
@@ -19,6 +19,7 @@
 package org.carbondata.query.carbon.aggregator.dimension.impl;
 
 import java.nio.ByteBuffer;
+import java.sql.Timestamp;
 
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
@@ -96,17 +97,19 @@ public class DirectDictionaryDimensionAggregator implements DimensionDataAggrega
       MeasureAggregator[] aggeragtor) {
     byte[] dimensionData = scannedResult.getDimensionKey(blockIndex);
     int surrogateKey = CarbonUtil.getSurrogateKey(dimensionData, buffer);
-    Object dataBasedOnDataType =
-        (long) directDictionaryGenerator.getValueFromSurrogate(surrogateKey) / 1000;
+    Object valueFromSurrogate = directDictionaryGenerator.getValueFromSurrogate(surrogateKey);
+    if (null != valueFromSurrogate) {
+      Timestamp dataBasedOnDataType = new Timestamp((long) valueFromSurrogate / 1000);
 
-    if (actualTypeAggregatorIndex.length > 0) {
-      for (int j = 0; j < actualTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + actualTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
+      if (actualTypeAggregatorIndex.length > 0) {
+        for (int j = 0; j < actualTypeAggregatorIndex.length; j++) {
+          aggeragtor[aggregatorStartIndex + actualTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
+        }
       }
-    }
-    if (numberTypeAggregatorIndex.length > 0) {
-      for (int j = 0; j < numberTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + numberTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
+      if (numberTypeAggregatorIndex.length > 0) {
+        for (int j = 0; j < numberTypeAggregatorIndex.length; j++) {
+          aggeragtor[aggregatorStartIndex + numberTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
+        }
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/91951acf/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
index 4897736..59ef1be 100644
--- a/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
@@ -24,6 +24,7 @@ import java.util.SortedMap;
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
 import org.carbondata.query.expression.ColumnExpression;
@@ -80,9 +81,11 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
         // column expression.
         // we need to check if the other expression contains column
         // expression or not in depth.
-        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)||
-            FilterUtil.isExpressionNeedsToResolved(rightExp,isIncludeFilter) &&
-            columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)){
+        CarbonDimension dimension = columnExpression.getDimension();
+        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)
+            || FilterUtil.isExpressionNeedsToResolved(rightExp, isIncludeFilter) &&
+            dimension.hasEncoding(Encoding.DICTIONARY) && !dimension
+            .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
           isExpressionResolve = true;
         } else {
           //Visitor pattern is been used in this scenario inorder to populate the

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/91951acf/integration/spark/src/test/resources/datasamplenull.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datasamplenull.csv b/integration/spark/src/test/resources/datasamplenull.csv
new file mode 100644
index 0000000..475bbe1
--- /dev/null
+++ b/integration/spark/src/test/resources/datasamplenull.csv
@@ -0,0 +1,3 @@
+ID,date,country,name,phonetype,serialname,salary
+1,2015/7/23,china,aaa1,phone197,ASD69643,15000
+2,,china,aaa2,phone756,ASD42892,15001
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/91951acf/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeNullDataTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeNullDataTest.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeNullDataTest.scala
new file mode 100644
index 0000000..c8c1f81
--- /dev/null
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/directdictionary/TimestampDataTypeNullDataTest.scala
@@ -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.carbondata.spark.testsuite.directdictionary
+
+import java.io.File
+import java.sql.Timestamp
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.apache.spark.sql.hive.HiveContext
+import org.carbondata.core.constants.CarbonCommonConstants
+import org.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampGranularityConstants
+import org.carbondata.core.util.CarbonProperties
+import org.scalatest.BeforeAndAfterAll
+
+
+/**
+  * Test Class for detailed query on timestamp datatypes
+  *
+  *
+  */
+class TimestampDataTypeNullDataTest extends QueryTest with BeforeAndAfterAll {
+  var oc: HiveContext = _
+
+  override def beforeAll {
+    try {
+      CarbonProperties.getInstance()
+        .addProperty(TimeStampGranularityConstants.CARBON_CUTOFF_TIMESTAMP, "2000-12-13 02:10.00.0")
+      CarbonProperties.getInstance()
+        .addProperty(TimeStampGranularityConstants.CARBON_TIME_GRANULARITY,
+          TimeStampGranularityConstants.TIME_GRAN_SEC.toString
+        )
+      sql(
+        """CREATE TABLE IF NOT EXISTS timestampTyeNullData
+                     (ID Int, date Timestamp, country String,
+                     name String, phonetype String, serialname String, salary Int)
+                    STORED BY 'org.apache.carbondata.format'"""
+      )
+
+      CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/mm/dd")
+      val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
+        .getCanonicalPath
+      var csvFilePath = currentDirectory + "/src/test/resources/datasamplenull.csv"
+      sql("LOAD DATA LOCAL INPATH '" + csvFilePath + "' INTO TABLE timestampTyeNullData").collect();
+
+    } catch {
+      case x: Throwable => CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+    }
+  }
+
+  test("SELECT max(date) FROM timestampTyeNullData where date is not null") {
+    checkAnswer(
+      sql("SELECT max(date) FROM timestampTyeNullData where date is not null"),
+      Seq(Row(Timestamp.valueOf("2015-01-23 00:07:00.0"))
+      )
+    )
+  }
+    test("SELECT * FROM timestampTyeNullData where date is null") {
+      checkAnswer(
+        sql("SELECT date FROM timestampTyeNullData where date is null"),
+        Seq(Row(null)
+        ))
+  }
+
+  override def afterAll {
+    sql("drop table timestampTyeNullData")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+    CarbonProperties.getInstance().addProperty("carbon.direct.dictionary", "false")
+  }
+
+}
\ No newline at end of file


[15/56] [abbrv] incubator-carbondata git commit: Refactored core package and fixed all testcases (#684)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 600519f..4c7c460 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.errors.attachTree
 import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.execution.{SparkPlan, UnaryNode}
-import org.apache.spark.sql.hive.CarbonMetastoreCatalog
+import org.apache.spark.sql.hive.{CarbonMetastoreCatalog, CarbonMetastoreTypes}
 import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
 import org.apache.spark.sql.types._
 import org.apache.spark.unsafe.types.UTF8String
@@ -54,20 +54,19 @@ case class CarbonDictionaryDecoder(
     child.output.map { a =>
       val attr = aliasMap.getOrElse(a, a)
       val relation = relations.find(p => p.contains(attr))
-      if(relation.isDefined) {
+      if(relation.isDefined && canBeDecoded(attr)) {
         val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
         val carbonDimension = carbonTable
           .getDimensionByName(carbonTable.getFactTableName, attr.name)
         if (carbonDimension != null &&
             carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
-            !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
-            canBeDecoded(attr)) {
+            !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
           val newAttr = AttributeReference(a.name,
-            convertCarbonToSparkDataType(carbonDimension),
+            convertCarbonToSparkDataType(carbonDimension,
+              relation.get.carbonRelation.carbonRelation),
             a.nullable,
             a.metadata)(a.exprId,
             a.qualifiers).asInstanceOf[Attribute]
-          newAttr.resolved
           newAttr
         } else {
           a
@@ -89,15 +88,29 @@ case class CarbonDictionaryDecoder(
     }
   }
 
-  def convertCarbonToSparkDataType(carbonDimension: CarbonDimension): types.DataType = {
+  def convertCarbonToSparkDataType(carbonDimension: CarbonDimension,
+      relation: CarbonRelation): types.DataType = {
     carbonDimension.getDataType match {
       case DataType.STRING => StringType
       case DataType.INT => IntegerType
       case DataType.LONG => LongType
       case DataType.DOUBLE => DoubleType
       case DataType.BOOLEAN => BooleanType
-      case DataType.DECIMAL => DecimalType.DoubleDecimal
+      case DataType.DECIMAL =>
+        val scale: Int = carbonDimension.getColumnSchema.getScale
+        val precision: Int = carbonDimension.getColumnSchema.getPrecision
+        if (scale > 0 && precision > 0)  {
+          DecimalType(scale, precision)
+        } else {
+          DecimalType(18, 2)
+        }
       case DataType.TIMESTAMP => TimestampType
+      case DataType.STRUCT =>
+        CarbonMetastoreTypes
+        .toDataType(s"struct<${ relation.getStructChildren(carbonDimension.getColName) }>")
+      case DataType.ARRAY =>
+        CarbonMetastoreTypes
+        .toDataType(s"array<${ relation.getArrayChildren(carbonDimension.getColName) }>")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index cb20246..ba4c37e 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -26,24 +26,20 @@ import org.apache.hadoop.conf.Configuration
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.catalyst.InternalRow
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.LeafNode
 import org.apache.spark.sql.hive.CarbonMetastoreCatalog
-import org.apache.spark.sql.types.{DataType, Decimal}
 import org.apache.spark.unsafe.types.UTF8String
 
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.util.CarbonProperties
 import org.carbondata.query.carbon.model._
-import org.carbondata.spark.{CarbonFilters, RawKey, RawKeyImpl}
+import org.carbondata.spark.{CarbonFilters, RawValue, RawValueImpl}
 import org.carbondata.spark.rdd.CarbonScanRDD
 
 case class CarbonScan(
     var attributesRaw: Seq[Attribute],
     relationRaw: CarbonRelation,
-    dimensionPredicatesRaw: Seq[Expression],
-    aggExprsRaw: Option[Seq[Expression]],
-    useBinaryAggregator: Boolean)(@transient val ocRaw: SQLContext) extends LeafNode {
+    dimensionPredicatesRaw: Seq[Expression])(@transient val ocRaw: SQLContext) extends LeafNode {
   val carbonTable = relationRaw.metaData.carbonTable
   val selectedDims = scala.collection.mutable.MutableList[QueryDimension]()
   val selectedMsrs = scala.collection.mutable.MutableList[QueryMeasure]()
@@ -95,24 +91,6 @@ case class CarbonScan(
           }
         }
       }
-    // Just find out that any aggregation functions are present on dimensions.
-    aggExprsRaw match {
-      case Some(aggExprs) =>
-        aggExprs.foreach {
-          case Alias(agg: AggregateExpression, name) =>
-            agg.collect {
-              case attr: AttributeReference =>
-                val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-                if(dims.nonEmpty) {
-                  plan.addAggDimAggInfo(dims.head.getColumnName,
-                    dims.head.getAggregateFunction,
-                    dims.head.getQueryOrder)
-                }
-            }
-          case _ =>
-        }
-      case _ =>
-    }
 
     // Fill the selected dimensions & measures obtained from
     // attributes to query plan  for detailed query
@@ -179,14 +157,13 @@ case class CarbonScan(
   }
 
 
-  def inputRdd: CarbonScanRDD[Array[Any], Any] = {
+  def inputRdd: CarbonScanRDD[Array[Any]] = {
 
     val conf = new Configuration()
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    buildCarbonPlan.getDimAggregatorInfos.clear()
     val model = QueryModel.createModel(
       absoluteTableIdentifier, buildCarbonPlan, carbonTable)
-    val kv: RawKey[Array[Any], Any] = new RawKeyImpl()
+    val kv: RawValue[Array[Any]] = new RawValueImpl
     // setting queryid
     buildCarbonPlan.setQueryId(ocRaw.getConf("queryId", System.nanoTime() + ""))
 
@@ -224,9 +201,9 @@ case class CarbonScan(
 
         override def next(): InternalRow =
           if (outUnsafeRows) {
-            unsafeProjection(new GenericMutableRow(iter.next()._1.map(toType)))
+            unsafeProjection(new GenericMutableRow(iter.next().map(toType)))
           } else {
-            new GenericMutableRow(iter.next()._1.map(toType))
+            new GenericMutableRow(iter.next().map(toType))
           }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSQLConf.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSQLConf.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSQLConf.scala
index 96a86a7..6ed8c0d 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSQLConf.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSQLConf.scala
@@ -20,13 +20,6 @@ package org.apache.spark.sql
 import org.apache.spark.sql.SQLConf.SQLConfEntry
 import org.apache.spark.sql.hive.CarbonSQLDialect
 
-object CarbonSQLConf {
-
-  val PUSH_COMPUTATION = SQLConfEntry.booleanConf("spark.sql.carbon.push.computation",
-    defaultValue = Some(true))
-
-}
-
  /**
   * A trait that enables the setting and getting of mutable config parameters/hints.
   *
@@ -40,8 +33,4 @@ class CarbonSQLConf extends SQLConf {
 
   override def caseSensitiveAnalysis: Boolean = getConf(SQLConf.CASE_SENSITIVE, false)
 
-  import CarbonSQLConf._
-
-  private[sql] def pushComputation: Boolean = getConf(PUSH_COMPUTATION)
-
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index a72aaa1..9239cec 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -170,7 +170,7 @@ class CarbonSqlParser()
     initLexical
     phrase(start)(new lexical.Scanner(input)) match {
       case Success(plan, _) => plan match {
-        case x: LoadCube =>
+        case x: LoadTable =>
           x.inputSqlString = input
           x
         case logicalPlan => logicalPlan
@@ -969,7 +969,7 @@ class CarbonSqlParser()
 
         }
         val patitionOptionsMap = partionDataOptions.toMap
-        LoadCube(schema, cubename, filePath, dimFolderPath.getOrElse(Seq()),
+        LoadTable(schema, cubename, filePath, dimFolderPath.getOrElse(Seq()),
             patitionOptionsMap, false)
     }
 
@@ -985,7 +985,7 @@ class CarbonSqlParser()
             validateOptions(partionDataOptions)
           }
           val patitionOptionsMap = partionDataOptions.getOrElse(List.empty[(String, String)]).toMap
-          LoadCube(schema, cubename, filePath, Seq(), patitionOptionsMap, isOverwrite.isDefined)
+          LoadTable(schema, cubename, filePath, Seq(), patitionOptionsMap, isOverwrite.isDefined)
       }
 
   private def validateOptions(partionDataOptions: Option[List[(String, String)]]): Unit = {
@@ -1079,9 +1079,7 @@ class CarbonSqlParser()
       opt(";") ^^ {
       case tabletype ~ exists ~ schemaName ~ resourceName =>
         tabletype match {
-          case agg ~ table =>
-            DropAggregateTableCommand(exists.isDefined, schemaName, resourceName.toLowerCase())
-          case _ => DropCubeCommand(exists.isDefined, schemaName, resourceName.toLowerCase())
+          case _ => DropTableCommand(exists.isDefined, schemaName, resourceName.toLowerCase())
         }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index fcc2e8a..e36d148 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -1381,7 +1381,7 @@ private[sql] case class DeleteLoadsByLoadDate(
 
 }
 
-private[sql] case class LoadCube(
+private[sql] case class LoadTable(
     schemaNameOp: Option[String],
     tableName: String,
     factPathFromUser: String,
@@ -1693,7 +1693,7 @@ private[sql] case class MergeTable(dbName: String, cubeName: String, tableName:
   }
 }
 
-private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Option[String],
+private[sql] case class DropTableCommand(ifExistsSet: Boolean, schemaNameOp: Option[String],
     tableName: String)
   extends RunnableCommand {
 
@@ -1755,7 +1755,7 @@ private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Opti
                 Some(relation.cubeMeta.carbonTableIdentifier.getDatabaseName))
               )(sqlContext)
           CarbonDataRDDFactory
-            .dropCube(sqlContext.sparkContext, dbName, tableName,
+            .dropTable(sqlContext.sparkContext, dbName, tableName,
               relation.cubeMeta.partitioner)
           QueryPartitionHelper.getInstance().removePartition(dbName, tableName)
 
@@ -1789,33 +1789,6 @@ private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Opti
   }
 }
 
-private[sql] case class DropAggregateTableCommand(ifExistsSet: Boolean,
-    schemaNameOp: Option[String],
-    tableName: String) extends RunnableCommand {
-
-  def run(sqlContext: SQLContext): Seq[Row] = {
-    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
-    val identifier = TableIdentifier(tableName, Option(dbName))
-    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
-      .lookupRelation1(identifier)(sqlContext).asInstanceOf[CarbonRelation]
-
-    if (relation == null) {
-      if (!ifExistsSet) {
-        sys.error(s"Aggregate Table $dbName.$tableName does not exist")
-      }
-    }
-    else {
-      CarbonDataRDDFactory.dropAggregateTable(
-        sqlContext.sparkContext,
-        dbName,
-        tableName,
-        relation.cubeMeta.partitioner)
-    }
-
-    Seq.empty
-  }
-}
-
 private[sql] case class ShowLoads(
     schemaNameOp: Option[String],
     tableName: String,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
index 829c487..76edc11 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
@@ -33,7 +33,6 @@ import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, DropTable,
 import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
 
 import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 
 
@@ -64,10 +63,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
             carbonRawScan(projectList,
               predicates,
               carbonRelation,
-              l,
-              None,
-              detailQuery = true,
-              useBinaryAggregation = false)(sqlContext)._1 :: Nil
+              l)(sqlContext) :: Nil
           }
         case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
           CarbonDictionaryDecoder(relations,
@@ -85,10 +81,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
     private def carbonRawScan(projectList: Seq[NamedExpression],
       predicates: Seq[Expression],
       relation: CarbonDatasourceRelation,
-      logicalRelation: LogicalRelation,
-      groupExprs: Option[Seq[Expression]],
-      detailQuery: Boolean,
-      useBinaryAggregation: Boolean)(sc: SQLContext): (SparkPlan, Boolean) = {
+      logicalRelation: LogicalRelation)(sc: SQLContext): SparkPlan = {
 
       val tableName: String =
         relation.carbonRelation.metaData.carbonTable.getFactTableName.toLowerCase
@@ -97,49 +90,32 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
       val projectSet = AttributeSet(projectList.flatMap(_.references))
       val scan = CarbonScan(projectSet.toSeq,
         relation.carbonRelation,
-        predicates,
-        groupExprs,
-        useBinaryAggregation)(sqlContext)
-      val dimAggrsPresence: Boolean = scan.buildCarbonPlan.getDimAggregatorInfos.size() > 0
+        predicates)(sqlContext)
       projectList.map {
         case attr: AttributeReference =>
         case Alias(attr: AttributeReference, _) =>
         case others =>
-          others.references
-              .map(f => scan.attributesNeedToDecode.add(f.asInstanceOf[AttributeReference]))
-      }
-      if (!detailQuery) {
-        if (scan.attributesNeedToDecode.size > 0) {
-          val decoder = getCarbonDecoder(logicalRelation,
-            sc,
-            tableName,
-            scan.attributesNeedToDecode.asScala.toSeq,
-            scan)
-          if (scan.unprocessedExprs.nonEmpty) {
-            val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
-            (Project(projectList, filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)), true)
-          } else {
-            (Project(projectList, decoder), true)
+          others.references.map{f =>
+            val dictionary = relation.carbonRelation.metaData.dictionaryMap.get(f.name)
+            if (dictionary.isDefined && dictionary.get) {
+              scan.attributesNeedToDecode.add(f.asInstanceOf[AttributeReference])
+            }
           }
+      }
+      if (scan.attributesNeedToDecode.size() > 0) {
+        val decoder = getCarbonDecoder(logicalRelation,
+          sc,
+          tableName,
+          scan.attributesNeedToDecode.asScala.toSeq,
+          scan)
+        if (scan.unprocessedExprs.nonEmpty) {
+          val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
+          Project(projectList, filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder))
         } else {
-          (scan, dimAggrsPresence)
+          Project(projectList, decoder)
         }
       } else {
-        if (scan.attributesNeedToDecode.size() > 0) {
-          val decoder = getCarbonDecoder(logicalRelation,
-            sc,
-            tableName,
-            scan.attributesNeedToDecode.asScala.toSeq,
-            scan)
-          if (scan.unprocessedExprs.nonEmpty) {
-            val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
-            (Project(projectList, filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)), true)
-          } else {
-            (Project(projectList, decoder), true)
-          }
-        } else {
-          (Project(projectList, scan), dimAggrsPresence)
-        }
+        Project(projectList, scan)
       }
     }
 
@@ -158,9 +134,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
       val projectExprsNeedToDecode = new java.util.HashSet[Attribute]()
       val scan = CarbonScan(projectList.map(_.toAttribute),
         relation.carbonRelation,
-        predicates,
-        None,
-        useBinaryAggregator = false)(sqlContext)
+        predicates)(sqlContext)
       projectExprsNeedToDecode.addAll(scan.attributesNeedToDecode)
       if (projectExprsNeedToDecode.size() > 0) {
         val decoder = getCarbonDecoder(logicalRelation,
@@ -206,18 +180,6 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         case _ => false
       }
     }
-
-    private def isGroupByPresentOnMeasures(groupingExpressions: Seq[Expression],
-      carbonTable: CarbonTable): Boolean = {
-      groupingExpressions.map { g =>
-        g.collect {
-          case attr: AttributeReference
-            if carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) != null =>
-            return true
-        }
-      }
-      false
-    }
   }
 
   object DDLStrategies extends Strategy {
@@ -233,17 +195,17 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
       case DropTable(tableName, ifNotExists)
         if CarbonEnv.getInstance(sqlContext).carbonCatalog
             .tableExists(TableIdentifier(tableName.toLowerCase))(sqlContext) =>
-        ExecutedCommand(DropCubeCommand(ifNotExists, None, tableName.toLowerCase)) :: Nil
+        ExecutedCommand(DropTableCommand(ifNotExists, None, tableName.toLowerCase)) :: Nil
       case ShowAggregateTablesCommand(schemaName) =>
         ExecutedCommand(ShowAggregateTables(schemaName, plan.output)) :: Nil
       case ShowLoadsCommand(schemaName, cube, limit) =>
         ExecutedCommand(ShowLoads(schemaName, cube, limit, plan.output)) :: Nil
-      case LoadCube(schemaNameOp, cubeName, factPathFromUser, dimFilesPath,
+      case LoadTable(schemaNameOp, cubeName, factPathFromUser, dimFilesPath,
       partionValues, isOverwriteExist, inputSqlString) =>
         val isCarbonTable = CarbonEnv.getInstance(sqlContext).carbonCatalog
             .tableExists(TableIdentifier(cubeName, schemaNameOp))(sqlContext)
         if (isCarbonTable || partionValues.nonEmpty) {
-          ExecutedCommand(LoadCube(schemaNameOp, cubeName, factPathFromUser,
+          ExecutedCommand(LoadTable(schemaNameOp, cubeName, factPathFromUser,
             dimFilesPath, partionValues, isOverwriteExist, inputSqlString)) :: Nil
         } else {
           ExecutedCommand(HiveNativeCommand(inputSqlString)) :: Nil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonDecoderOptimizerHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonDecoderOptimizerHelper.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonDecoderOptimizerHelper.scala
index 03bb23e..0f583d0 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonDecoderOptimizerHelper.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonDecoderOptimizerHelper.scala
@@ -55,17 +55,13 @@ class CarbonDecoderProcessor {
       case cd: CarbonDictionaryTempDecoder =>
         nodeList.add(Node(cd))
         process(cd.child, nodeList)
-      case j: Join =>
+      case j: BinaryNode =>
         val leftList = new util.ArrayList[AbstractNode]
         val rightList = new util.ArrayList[AbstractNode]
         nodeList.add(JoinNode(leftList, rightList))
         process(j.left, leftList)
         process(j.right, rightList)
-      case p: Project => process(p.child, nodeList)
-      case f: Filter => process(f.child, nodeList)
-      case s: Sort => process(s.child, nodeList)
-      case a: Aggregate => process(a.child, nodeList)
-      case l: Limit => process(l.child, nodeList)
+      case e: UnaryNode => process(e.child, nodeList)
       case _ =>
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
index d80c065..73cb3d5 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
@@ -44,13 +44,9 @@ class CarbonOptimizer(optimizer: Optimizer, conf: CatalystConf)
 
   override def execute(plan: LogicalPlan): LogicalPlan = {
     val executedPlan: LogicalPlan = optimizer.execute(plan)
-    if (!conf.asInstanceOf[CarbonSQLConf].pushComputation) {
-      val relations = collectCarbonRelation(plan)
-      if (relations.nonEmpty) {
-        new ResolveCarbonFunctions(relations)(executedPlan)
-      } else {
-        executedPlan
-      }
+    val relations = collectCarbonRelation(plan)
+    if (relations.nonEmpty) {
+      new ResolveCarbonFunctions(relations)(executedPlan)
     } else {
       executedPlan
     }
@@ -111,19 +107,28 @@ class CarbonOptimizer(optimizer: Optimizer, conf: CatalystConf)
 
           case agg: Aggregate if !agg.child.isInstanceOf[CarbonDictionaryTempDecoder] =>
             val attrsOndimAggs = new util.HashSet[Attribute]
-            agg.aggregateExpressions.map { aggExp =>
-              aggExp.transform {
-                case aggExp: AggregateExpression =>
-                  collectDimensionAggregates(aggExp, attrsOndimAggs, aliasMap)
-                  aggExp
-                case a@Alias(attr: Attribute, name) =>
-                  aliasMap.put(a.toAttribute, attr)
-                  a
-              }
+            agg.aggregateExpressions.map {
+              case attr: AttributeReference =>
+              case a@Alias(attr: AttributeReference, name) => aliasMap.put(a.toAttribute, attr)
+              case aggExp: AggregateExpression =>
+                aggExp.transform {
+                  case aggExp: AggregateExpression =>
+                    collectDimensionAggregates(aggExp, attrsOndimAggs, aliasMap)
+                    aggExp
+                  case a@Alias(attr: Attribute, name) =>
+                    aliasMap.put(a.toAttribute, attr)
+                    a
+                }
+              case others =>
+                others.collect {
+                  case attr: AttributeReference
+                    if isDictionaryEncoded(attr, relations, aliasMap) =>
+                    attrsOndimAggs.add(aliasMap.getOrElse(attr, attr))
+                }
             }
             var child = agg.child
             // Incase if the child also aggregate then push down decoder to child
-            if (attrsOndimAggs.size() > 0 && !(child.equals(agg))) {
+            if (attrsOndimAggs.size() > 0 && !child.equals(agg)) {
               child = CarbonDictionaryTempDecoder(attrsOndimAggs,
                 new util.HashSet[Attribute](),
                 agg.child)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
index aee375a..cf31a7b 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
@@ -105,14 +105,19 @@ object CarbonFilters {
   def selectFilters(filters: Seq[Expression],
       attrList: java.util.HashSet[Attribute],
       aliasMap: CarbonAliasDecoderRelation): Unit = {
-    def translate(expr: Expression): Option[sources.Filter] = {
+    def translate(expr: Expression, or: Boolean = false): Option[sources.Filter] = {
       expr match {
-        case Or(left, right) =>
-          for {
-            leftFilter <- translate(left)
-            rightFilter <- translate(right)
-          } yield {
-            sources.Or(leftFilter, rightFilter)
+        case or@ Or(left, right) =>
+
+          val leftFilter = translate(left, true)
+          val rightFilter = translate(right, true)
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            Some( sources.Or(leftFilter.get, rightFilter.get))
+          } else {
+            or.collect {
+              case attr: AttributeReference => attrList.add(aliasMap.getOrElse(attr, attr))
+            }
+            None
           }
 
         case And(left, right) =>
@@ -151,28 +156,35 @@ object CarbonFilters {
           Some(sources.In(a.name, hSet.toArray))
 
         case others =>
-          others.collect {
-            case attr: AttributeReference =>
-              attrList.add(aliasMap.getOrElse(attr, attr))
+          if (!or) {
+            others.collect {
+              case attr: AttributeReference =>
+                attrList.add(aliasMap.getOrElse(attr, attr))
+            }
           }
           None
       }
     }
-    filters.flatMap(translate).toArray
+    filters.flatMap(translate(_, false)).toArray
   }
 
   def processExpression(exprs: Seq[Expression],
       attributesNeedToDecode: java.util.HashSet[AttributeReference],
       unprocessedExprs: ArrayBuffer[Expression],
       carbonTable: CarbonTable): Option[CarbonExpression] = {
-    def transformExpression(expr: Expression): Option[CarbonExpression] = {
+    def transformExpression(expr: Expression, or: Boolean = false): Option[CarbonExpression] = {
       expr match {
-        case Or(left, right) =>
-          for {
-            leftFilter <- transformExpression(left)
-            rightFilter <- transformExpression(right)
-          } yield {
-            new OrExpression(leftFilter, rightFilter)
+        case or@ Or(left, right) =>
+          val leftFilter = transformExpression(left, true)
+          val rightFilter = transformExpression(right, true)
+          if (leftFilter.isDefined && rightFilter.isDefined) {
+            Some(new OrExpression(leftFilter.get, rightFilter.get))
+          } else {
+            or.collect {
+              case attr: AttributeReference => attributesNeedToDecode.add(attr)
+            }
+            unprocessedExprs += or
+            None
           }
 
         case And(left, right) =>
@@ -220,14 +232,16 @@ object CarbonFilters {
             CarbonLiteralExpression(name, CarbonScalaUtil.convertSparkToCarbonDataType(dataType)))
         case Cast(left, right) if !left.isInstanceOf[Literal] => transformExpression(left)
         case others =>
-          others.collect {
-            case attr: AttributeReference => attributesNeedToDecode.add(attr)
+          if (!or) {
+            others.collect {
+              case attr: AttributeReference => attributesNeedToDecode.add(attr)
+            }
+            unprocessedExprs += others
           }
-          unprocessedExprs += others
           None
       }
     }
-    exprs.flatMap(transformExpression).reduceOption(new AndExpression(_, _))
+    exprs.flatMap(transformExpression(_, false)).reduceOption(new AndExpression(_, _))
   }
 
   private def getActualCarbonDataType(column: String, carbonTable: CarbonTable) = {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
index 6ee882b..cb87818 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
@@ -26,35 +26,23 @@
 package org.carbondata.spark
 
 import org.carbondata.core.load.LoadMetadataDetails
-import org.carbondata.query.carbon.result.BatchRawResult
-import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
 
-trait KeyVal[K, V] extends Serializable {
-  def getKey(key: CarbonKey, value: CarbonValue): (K, V)
-
-}
-
-class KeyValImpl extends KeyVal[CarbonKey, CarbonValue] {
-  override def getKey(key: CarbonKey, value: CarbonValue): (CarbonKey, CarbonValue) = (key, value)
+trait Value[V] extends Serializable {
+  def getValue(value: Array[Object]): V
 }
 
-trait RawKeyVal[K, V] extends Serializable {
-  def getKey(key: BatchRawResult, value: Any): (K, V)
-
+class ValueImpl extends Value[Array[Object]] {
+  override def getValue(value: Array[Object]): Array[Object] = value
 }
 
-class RawKeyValImpl extends RawKeyVal[BatchRawResult, Any] {
-  override def getKey(key: BatchRawResult, value: Any): (BatchRawResult, Any) = (key, value)
+trait RawValue[V] extends Serializable {
+  def getValue(value: Array[Any]): V
 }
 
-trait RawKey[K, V] extends Serializable {
-  def getKey(key: Array[Any], value: Any): (K, V)
-
+class RawValueImpl extends RawValue[Array[Any]] {
+  override def getValue(value: Array[Any]): Array[Any] = value
 }
 
-class RawKeyImpl extends RawKey[Array[Any], Any] {
-  override def getKey(key: Array[Any], value: Any): (Array[Any], Any) = (key, value)
-}
 trait Result[K, V] extends Serializable {
   def getKey(key: Int, value: LoadMetadataDetails): (K, V)
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonCleanFilesRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
index 4e71be1..17d83b2 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonCleanFilesRDD.scala
@@ -18,23 +18,23 @@
 package org.carbondata.spark.rdd
 
 import scala.collection.JavaConverters._
+import scala.reflect.ClassTag
 
 import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.execution.command.Partitioner
 
-import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
-import org.carbondata.spark.KeyVal
+import org.carbondata.spark.Value
 import org.carbondata.spark.util.CarbonQueryUtil
 
 
-class CarbonCleanFilesRDD[K, V](
+class CarbonCleanFilesRDD[V: ClassTag](
     sc: SparkContext,
-    keyClass: KeyVal[K, V],
+    valueClass: Value[V],
     schemaName: String,
     cubeName: String,
     partitioner: Partitioner)
-  extends RDD[(K, V)](sc, Nil) with Logging {
+  extends RDD[V](sc, Nil) with Logging {
 
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
 
@@ -43,8 +43,8 @@ class CarbonCleanFilesRDD[K, V](
     splits.zipWithIndex.map(s => new CarbonLoadPartition(id, s._2, s._1))
   }
 
-  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val iter = new Iterator[(K, V)] {
+  override def compute(theSplit: Partition, context: TaskContext): Iterator[V] = {
+    val iter = new Iterator[(V)] {
       val split = theSplit.asInstanceOf[CarbonLoadPartition]
       logInfo("Input split: " + split.serializableHadoopSplit.value)
       // TODO call CARBON delete API
@@ -61,14 +61,12 @@ class CarbonCleanFilesRDD[K, V](
         !finished
       }
 
-      override def next(): (K, V) = {
+      override def next(): V = {
         if (!hasNext) {
           throw new java.util.NoSuchElementException("End of stream")
         }
         havePair = false
-        val row = new CarbonKey(null)
-        val value = new CarbonValue(null)
-        keyClass.getKey(row, value)
+        valueClass.getValue(null)
       }
 
     }
@@ -78,7 +76,7 @@ class CarbonCleanFilesRDD[K, V](
   override def getPreferredLocations(split: Partition): Seq[String] = {
     val theSplit = split.asInstanceOf[CarbonLoadPartition]
     val s = theSplit.serializableHadoopSplit.value.getLocations.asScala
-    logInfo("Host Name : " + s(0) + s.length)
+    logInfo("Host Name : " + s.head + s.length)
     s
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 0672281..af2271f 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -29,23 +29,22 @@ import org.apache.hadoop.conf.{Configurable, Configuration}
 import org.apache.hadoop.mapreduce.Job
 import org.apache.hadoop.mapreduce.lib.input.FileSplit
 import org.apache.spark.{Logging, Partition, SparkContext}
-import org.apache.spark.sql.{CarbonEnv, CarbonRelation, SQLContext}
+import org.apache.spark.sql.{CarbonEnv, SQLContext}
 import org.apache.spark.sql.execution.command.{AlterTableModel, CompactionModel, Partitioner}
 import org.apache.spark.util.{FileUtils, SplitUtils}
 
 import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonDataLoadSchema, CarbonTableIdentifier}
+import org.carbondata.core.carbon.CarbonDataLoadSchema
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo
 import org.carbondata.core.carbon.metadata.CarbonMetadata
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
 import org.carbondata.core.locks.{CarbonLockFactory, LockUsage}
-import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.carbondata.core.util.CarbonUtil
 import org.carbondata.integration.spark.merger.CompactionType
 import org.carbondata.lcm.status.SegmentStatusManager
 import org.carbondata.processing.util.CarbonDataProcessorUtil
-import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
 import org.carbondata.spark._
 import org.carbondata.spark.load._
 import org.carbondata.spark.merger.CarbonDataMergerUtil
@@ -129,7 +128,7 @@ object CarbonDataRDDFactory extends Logging {
     if (-1 == currentRestructNumber) {
       currentRestructNumber = 0
     }
-    var segmentStatusManager = new SegmentStatusManager(cube.getAbsoluteTableIdentifier)
+    val segmentStatusManager = new SegmentStatusManager(cube.getAbsoluteTableIdentifier)
     val loadMetadataDetailsArray = segmentStatusManager.readLoadMetadata(cube.getMetaDataFilepath())
       .toList
     val resultMap = new CarbonDeleteLoadByDateRDD(
@@ -408,9 +407,8 @@ object CarbonDataRDDFactory extends Logging {
 
       // Check if any load need to be deleted before loading new data
       deleteLoadsAndUpdateMetadata(carbonLoadModel, carbonTable, partitioner, hdfsStoreLocation,
-        false,
-        currentRestructNumber
-      )
+        isForceDeletion = false,
+        currentRestructNumber)
       if (null == carbonLoadModel.getLoadMetadataDetails) {
         readLoadMetadataDetails(carbonLoadModel, hdfsStoreLocation)
       }
@@ -668,9 +666,9 @@ object CarbonDataRDDFactory extends Logging {
 
   def readLoadMetadataDetails(model: CarbonLoadModel, hdfsStoreLocation: String): Unit = {
     val metadataPath = model.getCarbonDataLoadSchema.getCarbonTable.getMetaDataFilepath
-    var segmentStatusManager = new SegmentStatusManager(model.getCarbonDataLoadSchema.getCarbonTable
-      .
-        getAbsoluteTableIdentifier)
+    val segmentStatusManager =
+      new SegmentStatusManager(
+        model.getCarbonDataLoadSchema.getCarbonTable.getAbsoluteTableIdentifier)
     val details = segmentStatusManager.readLoadMetadata(metadataPath)
     model.setLoadMetadataDetails(details.toList.asJava)
   }
@@ -704,22 +702,13 @@ object CarbonDataRDDFactory extends Logging {
     }
   }
 
-  def dropAggregateTable(
+  def dropTable(
       sc: SparkContext,
       schema: String,
       cube: String,
       partitioner: Partitioner) {
-    val kv: KeyVal[CarbonKey, CarbonValue] = new KeyValImpl()
-    new CarbonDropAggregateTableRDD(sc, kv, schema, cube, partitioner).collect
-  }
-
-  def dropCube(
-      sc: SparkContext,
-      schema: String,
-      cube: String,
-      partitioner: Partitioner) {
-    val kv: KeyVal[CarbonKey, CarbonValue] = new KeyValImpl()
-    new CarbonDropCubeRDD(sc, kv, schema, cube, partitioner).collect
+    val v: Value[Array[Object]] = new ValueImpl()
+    new CarbonDropTableRDD(sc, v, schema, cube, partitioner).collect
   }
 
   def cleanFiles(
@@ -735,7 +724,7 @@ object CarbonDataRDDFactory extends Logging {
     if (-1 == currentRestructNumber) {
       currentRestructNumber = 0
     }
-    var carbonLock = CarbonLockFactory
+    val carbonLock = CarbonLockFactory
       .getCarbonLockObj(cube.getMetaDataFilepath, LockUsage.METADATA_LOCK)
     try {
       if (carbonLock.lockWithRetries()) {
@@ -744,8 +733,7 @@ object CarbonDataRDDFactory extends Logging {
           partitioner,
           hdfsStoreLocation,
           isForceDeletion = true,
-          currentRestructNumber
-        )
+          currentRestructNumber)
       }
     }
     finally {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
index 9fc5f9e..4616ca9 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDeleteLoadRDD.scala
@@ -18,36 +18,34 @@
 package org.carbondata.spark.rdd
 
 import scala.collection.JavaConverters._
+import scala.reflect.ClassTag
 
 import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.execution.command.Partitioner
 
-import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
-import org.carbondata.spark.KeyVal
+import org.carbondata.spark.Value
 import org.carbondata.spark.util.CarbonQueryUtil
 
-class CarbonDeleteLoadRDD[K, V](
-                                 sc: SparkContext,
-                                 keyClass: KeyVal[K, V],
-                                 loadId: Int,
-                                 schemaName: String,
-                                 cubeName: String,
-                                 partitioner: Partitioner)
-  extends RDD[(K, V)](sc, Nil) with Logging {
+class CarbonDeleteLoadRDD[V: ClassTag](
+    sc: SparkContext,
+    valueClass: Value[V],
+    loadId: Int,
+    schemaName: String,
+    cubeName: String,
+    partitioner: Partitioner)
+  extends RDD[V](sc, Nil) with Logging {
   sc.setLocalProperty("spark.scheduler.pool", "DDL")
 
   override def getPartitions: Array[Partition] = {
     val splits = CarbonQueryUtil.getTableSplits(schemaName, cubeName, null, partitioner)
-    val result = new Array[Partition](splits.length)
-    for (i <- 0 until result.length) {
-      result(i) = new CarbonLoadPartition(id, i, splits(i))
+    splits.zipWithIndex.map {f =>
+      new CarbonLoadPartition(id, f._2, f._1)
     }
-    result
   }
 
-  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val iter = new Iterator[(K, V)] {
+  override def compute(theSplit: Partition, context: TaskContext): Iterator[V] = {
+    val iter = new Iterator[V] {
       val split = theSplit.asInstanceOf[CarbonLoadPartition]
       logInfo("Input split: " + split.serializableHadoopSplit.value)
       // TODO call CARBON delete API
@@ -57,20 +55,18 @@ class CarbonDeleteLoadRDD[K, V](
 
       override def hasNext: Boolean = {
         if (!finished && !havePair) {
-          finished = !false
+          finished = true
           havePair = !finished
         }
         !finished
       }
 
-      override def next(): (K, V) = {
+      override def next(): V = {
         if (!hasNext) {
           throw new java.util.NoSuchElementException("End of stream")
         }
         havePair = false
-        val row = new CarbonKey(null)
-        val value = new CarbonValue(null)
-        keyClass.getKey(row, value)
+        valueClass.getValue(null)
       }
 
     }
@@ -81,7 +77,7 @@ class CarbonDeleteLoadRDD[K, V](
   override def getPreferredLocations(split: Partition): Seq[String] = {
     val theSplit = split.asInstanceOf[CarbonLoadPartition]
     val s = theSplit.serializableHadoopSplit.value.getLocations.asScala
-    logInfo("Host Name : " + s(0) + s.length)
+    logInfo("Host Name : " + s.head + s.length)
     s
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropAggregateTableRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropAggregateTableRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropAggregateTableRDD.scala
deleted file mode 100644
index 1b57890..0000000
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropAggregateTableRDD.scala
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one or more
- * contributor license agreements.  See the NOTICE file distributed with
- * this work for additional information regarding copyright ownership.
- * The ASF licenses this file to You under the Apache License, Version 2.0
- * (the "License"); you may not use this file except in compliance with
- * the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing, software
- * distributed under the License is distributed on an "AS IS" BASIS,
- * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
- * See the License for the specific language governing permissions and
- * limitations under the License.
- */
-
-package org.carbondata.spark.rdd
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.execution.command.Partitioner
-
-import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
-import org.carbondata.spark.KeyVal
-import org.carbondata.spark.util.CarbonQueryUtil
-
-
-class CarbonDropAggregateTableRDD[K, V](
-    sc: SparkContext,
-    keyClass: KeyVal[K, V],
-    schemaName: String,
-    cubeName: String,
-    partitioner: Partitioner)
-  extends RDD[(K, V)](sc, Nil) with Logging {
-
-  sc.setLocalProperty("spark.scheduler.pool", "DDL")
-
-  override def getPartitions: Array[Partition] = {
-    val splits = CarbonQueryUtil.getTableSplits(schemaName, cubeName, null, partitioner)
-    splits.zipWithIndex.map { s =>
-      new CarbonLoadPartition(id, s._2, s._1)
-    }
-  }
-
-  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val iter = new Iterator[(K, V)] {
-      val split = theSplit.asInstanceOf[CarbonLoadPartition]
-      logInfo("Input split: " + split.serializableHadoopSplit.value)
-      // TODO call CARBON delete API
-
-      var havePair = false
-      var finished = false
-
-      override def hasNext: Boolean = {
-        if (!finished && !havePair) {
-          finished = true
-          havePair = !finished
-        }
-        !finished
-      }
-
-      override def next(): (K, V) = {
-        if (!hasNext) {
-          throw new java.util.NoSuchElementException("End of stream")
-        }
-        havePair = false
-        val row = new CarbonKey(null)
-        val value = new CarbonValue(null)
-        keyClass.getKey(row, value)
-      }
-    }
-    iter
-  }
-
-  override def getPreferredLocations(split: Partition): Seq[String] = {
-    val theSplit = split.asInstanceOf[CarbonLoadPartition]
-    val s = theSplit.serializableHadoopSplit.value.getLocations.asScala
-    logInfo("Host Name : " + s.head + s.length)
-    s
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropCubeRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropCubeRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropCubeRDD.scala
deleted file mode 100644
index d0bc5d1..0000000
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropCubeRDD.scala
+++ /dev/null
@@ -1,77 +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.carbondata.spark.rdd
-
-import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.execution.command.Partitioner
-
-import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
-import org.carbondata.spark.KeyVal
-import org.carbondata.spark.util.CarbonQueryUtil
-
-class CarbonDropCubeRDD[K, V](
-    sc: SparkContext,
-    keyClass: KeyVal[K, V],
-    schemaName: String,
-    cubeName: String,
-    partitioner: Partitioner)
-  extends RDD[(K, V)](sc, Nil) with Logging {
-
-  sc.setLocalProperty("spark.scheduler.pool", "DDL")
-
-  override def getPartitions: Array[Partition] = {
-    val splits = CarbonQueryUtil.getTableSplits(schemaName, cubeName, null, partitioner)
-    splits.zipWithIndex.map { s =>
-      new CarbonLoadPartition(id, s._2, s._1)
-    }
-  }
-
-  override def compute(theSplit: Partition, context: TaskContext): Iterator[(K, V)] = {
-
-    val iter = new Iterator[(K, V)] {
-      val split = theSplit.asInstanceOf[CarbonLoadPartition]
-
-      val partitionCount = partitioner.partitionCount
-      // TODO: Clear Btree from memory
-
-      var havePair = false
-      var finished = false
-
-      override def hasNext: Boolean = {
-        if (!finished && !havePair) {
-          finished = true
-          havePair = !finished
-        }
-        !finished
-      }
-
-      override def next(): (K, V) = {
-        if (!hasNext) {
-          throw new java.util.NoSuchElementException("End of stream")
-        }
-        havePair = false
-        val row = new CarbonKey(null)
-        val value = new CarbonValue(null)
-        keyClass.getKey(row, value)
-      }
-    }
-    iter
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropTableRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropTableRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropTableRDD.scala
new file mode 100644
index 0000000..513916c
--- /dev/null
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDropTableRDD.scala
@@ -0,0 +1,76 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.carbondata.spark.rdd
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.execution.command.Partitioner
+
+import org.carbondata.spark.Value
+import org.carbondata.spark.util.CarbonQueryUtil
+
+class CarbonDropTableRDD[V: ClassTag](
+    sc: SparkContext,
+    valueClass: Value[V],
+    schemaName: String,
+    cubeName: String,
+    partitioner: Partitioner)
+  extends RDD[V](sc, Nil) with Logging {
+
+  sc.setLocalProperty("spark.scheduler.pool", "DDL")
+
+  override def getPartitions: Array[Partition] = {
+    val splits = CarbonQueryUtil.getTableSplits(schemaName, cubeName, null, partitioner)
+    splits.zipWithIndex.map { s =>
+      new CarbonLoadPartition(id, s._2, s._1)
+    }
+  }
+
+  override def compute(theSplit: Partition, context: TaskContext): Iterator[V] = {
+
+    val iter = new Iterator[V] {
+      val split = theSplit.asInstanceOf[CarbonLoadPartition]
+
+      val partitionCount = partitioner.partitionCount
+      // TODO: Clear Btree from memory
+
+      var havePair = false
+      var finished = false
+
+      override def hasNext: Boolean = {
+        if (!finished && !havePair) {
+          finished = true
+          havePair = !finished
+        }
+        !finished
+      }
+
+      override def next(): V = {
+        if (!hasNext) {
+          throw new java.util.NoSuchElementException("End of stream")
+        }
+        havePair = false
+        valueClass.getValue(null)
+      }
+    }
+    iter
+  }
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
index 5cb33fa..dd2a10a 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -36,7 +36,6 @@ import org.carbondata.core.util.CarbonProperties
 import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
 import org.carbondata.integration.spark.merger.{CarbonCompactionExecutor, CarbonCompactionUtil,
 RowResultMerger}
-import org.carbondata.query.carbon.result.{RowResult}
 import org.carbondata.query.carbon.result.iterator.RawResultIterator
 import org.carbondata.spark.MergeResult
 import org.carbondata.spark.load.{CarbonLoaderUtil, CarbonLoadModel}
@@ -179,7 +178,7 @@ class CarbonMergerRDD[K, V](
     val absoluteTableIdentifier: AbsoluteTableIdentifier = new AbsoluteTableIdentifier(
       hdfsStoreLocation, new CarbonTableIdentifier(schemaName, factTableName, tableId)
     )
-    val (carbonInputFormat: CarbonInputFormat[RowResult], job: Job) =
+    val (carbonInputFormat: CarbonInputFormat[Array[Object]], job: Job) =
       QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
     val result = new util.ArrayList[Partition](defaultParallelism)
     val mapsOfNodeBlockMapping: util.List[util.Map[String, util.List[TableBlockInfo]]] = new

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
index a95ae27..6693108 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -21,6 +21,7 @@ package org.carbondata.spark.rdd
 import java.util
 
 import scala.collection.JavaConverters._
+import scala.reflect.ClassTag
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.mapreduce.Job
@@ -33,10 +34,10 @@ import org.carbondata.core.iterator.CarbonIterator
 import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
 import org.carbondata.query.carbon.executor.QueryExecutorFactory
 import org.carbondata.query.carbon.model.QueryModel
-import org.carbondata.query.carbon.result.{BatchRawResult, RowResult}
-import org.carbondata.query.carbon.result.iterator.ChunkRawRowIterartor
+import org.carbondata.query.carbon.result.BatchResult
+import org.carbondata.query.carbon.result.iterator.ChunkRowIterator
 import org.carbondata.query.expression.Expression
-import org.carbondata.spark.RawKey
+import org.carbondata.spark.RawValue
 import org.carbondata.spark.load.CarbonLoaderUtil
 import org.carbondata.spark.util.QueryPlanUtil
 
@@ -58,29 +59,29 @@ class CarbonSparkPartition(rddId: Int, val idx: Int,
   * CarbonData file, this RDD will leverage CarbonData's index information to do CarbonData file
   * level filtering in driver side.
   */
-class CarbonScanRDD[K, V](
+class CarbonScanRDD[V: ClassTag](
   sc: SparkContext,
   queryModel: QueryModel,
   filterExpression: Expression,
-  keyClass: RawKey[K, V],
+  keyClass: RawValue[V],
   @transient conf: Configuration,
   cubeCreationTime: Long,
   schemaLastUpdatedTime: Long,
   baseStoreLocation: String)
-  extends RDD[(K, V)](sc, Nil) with Logging {
+  extends RDD[V](sc, Nil) with Logging {
 
   val defaultParallelism = sc.defaultParallelism
 
   override def getPartitions: Array[Partition] = {
     val startTime = System.currentTimeMillis()
-    val (carbonInputFormat: CarbonInputFormat[RowResult], job: Job) =
+    val (carbonInputFormat: CarbonInputFormat[Array[Object]], job: Job) =
       QueryPlanUtil.createCarbonInputFormat(queryModel.getAbsoluteTableIdentifier)
 
     val result = new util.ArrayList[Partition](defaultParallelism)
     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
     // set filter resolver tree
     try {
-      var filterResolver = carbonInputFormat
+      val filterResolver = carbonInputFormat
         .getResolvedFilter(job.getConfiguration, filterExpression)
 
       CarbonInputFormat.setFilterPredicates(job.getConfiguration, filterResolver)
@@ -147,15 +148,15 @@ class CarbonScanRDD[K, V](
     result.toArray(new Array[Partition](result.size()))
   }
 
-   override def compute(thepartition: Partition, context: TaskContext): Iterator[(K, V)] = {
-     val LOGGER = LogServiceFactory.getLogService(this.getClass().getName());
-     val iter = new Iterator[(K, V)] {
+   override def compute(thepartition: Partition, context: TaskContext): Iterator[V] = {
+     val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+     val iter = new Iterator[V] {
        var rowIterator: CarbonIterator[Array[Any]] = _
        var queryStartTime: Long = 0
        try {
          val carbonSparkPartition = thepartition.asInstanceOf[CarbonSparkPartition]
          if(!carbonSparkPartition.tableBlockInfos.isEmpty) {
-           queryModel.setQueryId(queryModel.getQueryId() + "_" + carbonSparkPartition.idx)
+           queryModel.setQueryId(queryModel.getQueryId + "_" + carbonSparkPartition.idx)
            // fill table block info
            queryModel.setTableBlockInfos(carbonSparkPartition.tableBlockInfos)
            queryStartTime = System.currentTimeMillis
@@ -164,13 +165,13 @@ class CarbonScanRDD[K, V](
            logInfo("*************************" + carbonPropertiesFilePath)
            if (null == carbonPropertiesFilePath) {
              System.setProperty("carbon.properties.filepath",
-               System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties");
+               System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties")
            }
            // execute query
-           rowIterator = new ChunkRawRowIterartor(
-             QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel)
-                 .asInstanceOf[CarbonIterator[BatchRawResult]])
-                 .asInstanceOf[CarbonIterator[Array[Any]]]
+           rowIterator = new ChunkRowIterator(
+             QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel).
+               asInstanceOf[CarbonIterator[BatchResult]]).asInstanceOf[CarbonIterator[Array[Any]]]
+
          }
        } catch {
          case e: Exception =>
@@ -187,19 +188,18 @@ class CarbonScanRDD[K, V](
 
        override def hasNext: Boolean = {
          if (!finished && !havePair) {
-           finished = (null == rowIterator) || (!rowIterator.hasNext())
+           finished = (null == rowIterator) || (!rowIterator.hasNext)
            havePair = !finished
          }
          !finished
        }
 
-       override def next(): (K, V) = {
+       override def next(): V = {
          if (!hasNext) {
            throw new java.util.NoSuchElementException("End of stream")
          }
          havePair = false
-         val row = rowIterator.next()
-         keyClass.getKey(row, null)
+         keyClass.getValue(rowIterator.next())
        }
 
        logInfo("********************** Total Time Taken to execute the query in Carbon Side: " +

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/carbondata/spark/util/QueryPlanUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/util/QueryPlanUtil.scala b/integration/spark/src/main/scala/org/carbondata/spark/util/QueryPlanUtil.scala
index 81b6feb..321e0f8 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/util/QueryPlanUtil.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/util/QueryPlanUtil.scala
@@ -25,7 +25,6 @@ import org.apache.hadoop.mapreduce.lib.input.FileInputFormat
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier
 import org.carbondata.hadoop.CarbonInputFormat
-import org.carbondata.query.carbon.result.RowResult
 
 /**
  * All the utility functions for carbon plan creation
@@ -36,8 +35,8 @@ object QueryPlanUtil {
    * createCarbonInputFormat from query model
    */
   def createCarbonInputFormat(absoluteTableIdentifier: AbsoluteTableIdentifier) :
-  (CarbonInputFormat[RowResult], Job) = {
-    val carbonInputFormat = new CarbonInputFormat[RowResult]()
+  (CarbonInputFormat[Array[Object]], Job) = {
+    val carbonInputFormat = new CarbonInputFormat[Array[Object]]()
     val jobConf: JobConf = new JobConf(new Configuration)
     val job: Job = new Job(jobConf)
     FileInputFormat.addInputPath(job, new Path(absoluteTableIdentifier.getStorePath))

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
index 4edfb7a..c2a2277 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
@@ -64,6 +64,7 @@ class AllDataTypesTestCaseAggregate extends QueryTest with BeforeAndAfterAll {
       sql("select channelsId, Latest_DAY from Carbon_automation_test where count(channelsId) = 1").collect
     } catch {
       case ce: UnsupportedOperationException => ce.getMessage
+      case ce: Exception => ce.getMessage
     }
   }
   

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java b/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
index 18b2bfe..641d18a 100644
--- a/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
+++ b/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
@@ -84,7 +84,7 @@ public class ColGroupMinMaxTest {
 	    }
 	    setMinData(data[i]);
 	    setMaxData(data[i]);
-	    System.out.println(Arrays.toString(data[i]));
+//	    System.out.println(Arrays.toString(data[i]));
 	}
 	mdkeyData = new byte[1000][];
 	for (int i = 0; i < 1000; i++) {


[02/56] [abbrv] incubator-carbondata git commit: [Issue 618]Supported Spark 1.6 in Carbondata (#670)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
index 84f050e..30a1070 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
@@ -30,6 +30,7 @@ import scala.util.parsing.combinator.RegexParsers
 
 import org.apache.spark
 import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
 import org.apache.spark.sql.execution.command.{AggregateTableAttributes, Partitioner}
@@ -120,28 +121,13 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     false
   }
 
-  def lookupRelation1(
-      databaseName: Option[String],
-      tableName: String,
-      alias: Option[String] = None)(sqlContext: SQLContext): LogicalPlan = {
-    val db = databaseName match {
-      case Some(name) => name
-      case _ => null
-    }
-    if (db == null) {
-      lookupRelation2(Seq(tableName), alias)(sqlContext)
-    } else {
-      lookupRelation2(Seq(db, tableName), alias)(sqlContext)
-    }
-  }
-
-  override def lookupRelation(tableIdentifier: Seq[String],
+  override def lookupRelation(tableIdentifier: TableIdentifier,
       alias: Option[String] = None): LogicalPlan = {
     try {
       super.lookupRelation(tableIdentifier, alias)
     } catch {
       case s: java.lang.Exception =>
-        lookupRelation2(tableIdentifier, alias)(hive.asInstanceOf[SQLContext])
+        lookupRelation1(tableIdentifier, alias)(hive.asInstanceOf[SQLContext])
     }
   }
 
@@ -153,64 +139,34 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     cubeCreationTime
   }
 
+  def lookupRelation1(dbName: Option[String],
+      tableName: String)(sqlContext: SQLContext): LogicalPlan = {
+    lookupRelation1(TableIdentifier(tableName, dbName))(sqlContext)
+  }
 
-  def lookupRelation2(tableIdentifier: Seq[String],
+  def lookupRelation1(tableIdentifier: TableIdentifier,
       alias: Option[String] = None)(sqlContext: SQLContext): LogicalPlan = {
     checkSchemasModifiedTimeAndReloadCubes()
-    tableIdentifier match {
-      case Seq(schemaName, cubeName) =>
-        val cubes = metadata.cubesMeta.filter(
-          c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(schemaName) &&
-               c.carbonTableIdentifier.getTableName.equalsIgnoreCase(cubeName))
-        if (cubes.nonEmpty) {
-          CarbonRelation(schemaName, cubeName,
-            CarbonSparkUtil.createSparkMeta(cubes.head.carbonTable), cubes.head, alias)(sqlContext)
-        } else {
-          LOGGER.audit(s"Table Not Found: $schemaName $cubeName")
-          throw new NoSuchTableException
-        }
-      case Seq(cubeName) =>
-        val currentDatabase = getDB.getDatabaseName(None, sqlContext)
-        val cubes = metadata.cubesMeta.filter(
-          c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(currentDatabase) &&
-               c.carbonTableIdentifier.getTableName.equalsIgnoreCase(cubeName))
-        if (cubes.nonEmpty) {
-          CarbonRelation(currentDatabase, cubeName,
-            CarbonSparkUtil.createSparkMeta(cubes.head.carbonTable), cubes.head, alias)(sqlContext)
-        } else {
-          LOGGER.audit(s"Table Not Found: $cubeName")
-          throw new NoSuchTableException
-        }
-      case _ =>
-        LOGGER.audit(s"Table Not Found: $tableIdentifier")
-        throw new NoSuchTableException
-    }
-  }
-
-  def cubeExists(db: Option[String], tableName: String)(sqlContext: SQLContext): Boolean = {
-    if (db.isEmpty || db.get == null || db.get == "") {
-      cubeExists(Seq(tableName))(sqlContext)
+    val database = tableIdentifier.database.getOrElse(getDB.getDatabaseName(None, sqlContext))
+    val cubes = metadata.cubesMeta.filter(
+      c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(database) &&
+           c.carbonTableIdentifier.getTableName.equalsIgnoreCase(tableIdentifier.table))
+    if (cubes.nonEmpty) {
+      CarbonRelation(database, tableIdentifier.table,
+        CarbonSparkUtil.createSparkMeta(cubes.head.carbonTable), cubes.head, alias)(sqlContext)
     } else {
-      cubeExists(Seq(db.get, tableName))(sqlContext)
+      LOGGER.audit(s"Table Not Found: ${tableIdentifier.table}")
+      throw new NoSuchTableException
     }
   }
 
-  def cubeExists(tableIdentifier: Seq[String])(sqlContext: SQLContext): Boolean = {
+  def tableExists(tableIdentifier: TableIdentifier)(sqlContext: SQLContext): Boolean = {
     checkSchemasModifiedTimeAndReloadCubes()
-    tableIdentifier match {
-      case Seq(schemaName, cubeName) =>
-        val cubes = metadata.cubesMeta.filter(
-          c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(schemaName) &&
-               c.carbonTableIdentifier.getTableName.equalsIgnoreCase(cubeName))
-        cubes.nonEmpty
-      case Seq(cubeName) =>
-        val currentDatabase = getDB.getDatabaseName(None, sqlContext)
-        val cubes = metadata.cubesMeta.filter(
-          c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(currentDatabase) &&
-               c.carbonTableIdentifier.getTableName.equalsIgnoreCase(cubeName))
-        cubes.nonEmpty
-      case _ => false
-    }
+    val database = tableIdentifier.database.getOrElse(getDB.getDatabaseName(None, sqlContext))
+    val cubes = metadata.cubesMeta.filter(
+      c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(database) &&
+           c.carbonTableIdentifier.getTableName.equalsIgnoreCase(tableIdentifier.table))
+    cubes.nonEmpty
   }
 
   def loadMetadata(metadataPath: String): MetaData = {
@@ -338,7 +294,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
       dbName: String, tableName: String, partitioner: Partitioner)
     (sqlContext: SQLContext): String = {
 
-    if (cubeExists(Seq(dbName, tableName))(sqlContext)) {
+    if (tableExists(TableIdentifier(tableName, Some(dbName)))(sqlContext)) {
       sys.error(s"Table [$tableName] already exists under Database [$dbName]")
     }
 
@@ -506,7 +462,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
   /**
    * Shows all schemas which has Database name like
    */
-  def showSchemas(schemaLike: Option[String]): Seq[String] = {
+  def showDatabases(schemaLike: Option[String]): Seq[String] = {
     checkSchemasModifiedTimeAndReloadCubes()
     metadata.cubesMeta.map { c =>
       schemaLike match {
@@ -526,7 +482,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
   /**
    * Shows all cubes for given schema.
    */
-  def getCubes(databaseName: Option[String])(sqlContext: SQLContext): Seq[(String, Boolean)] = {
+  def getTables(databaseName: Option[String])(sqlContext: SQLContext): Seq[(String, Boolean)] = {
 
     val schemaName = databaseName
       .getOrElse(sqlContext.asInstanceOf[HiveContext].catalog.client.currentDatabase)
@@ -539,21 +495,25 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
   /**
    * Shows all cubes in all schemas.
    */
-  def getAllCubes()(sqlContext: SQLContext): Seq[(String, String)] = {
+  def getAllTables()(sqlContext: SQLContext): Seq[TableIdentifier] = {
     checkSchemasModifiedTimeAndReloadCubes()
-    metadata.cubesMeta
-      .map { c => (c.carbonTableIdentifier.getDatabaseName, c.carbonTableIdentifier.getTableName) }
+    metadata.cubesMeta.map { c =>
+        TableIdentifier(c.carbonTableIdentifier.getTableName,
+          Some(c.carbonTableIdentifier.getDatabaseName))
+    }
   }
 
-  def dropCube(partitionCount: Int, tableStorePath: String, schemaName: String, cubeName: String)
+  def dropTable(partitionCount: Int, tableStorePath: String, tableIdentifier: TableIdentifier)
     (sqlContext: SQLContext) {
-    if (!cubeExists(Seq(schemaName, cubeName))(sqlContext)) {
-      LOGGER.audit(s"Drop Table failed. Table with $schemaName.$cubeName does not exist")
-      sys.error(s"Table with $schemaName.$cubeName does not exist")
+    val dbName = tableIdentifier.database.get
+    val tableName = tableIdentifier.table
+    if (!tableExists(tableIdentifier)(sqlContext)) {
+      LOGGER.audit(s"Drop Table failed. Table with ${dbName}.$tableName does not exist")
+      sys.error(s"Table with $dbName.$tableName does not exist")
     }
 
     val carbonTable = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
-      .getCarbonTable(schemaName + "_" + cubeName)
+      .getCarbonTable(dbName + "_" + tableName)
 
     if (null != carbonTable) {
       val metadatFilePath = carbonTable.getMetaDataFilepath
@@ -561,12 +521,12 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
 
       if (FileFactory.isFileExist(metadatFilePath, fileType)) {
         val file = FileFactory.getCarbonFile(metadatFilePath, fileType)
-        CarbonUtil.renameCubeForDeletion(partitionCount, tableStorePath, schemaName, cubeName)
+        CarbonUtil.renameCubeForDeletion(partitionCount, tableStorePath, dbName, tableName)
         CarbonUtil.deleteFoldersAndFilesSilent(file.getParentFile)
       }
 
       val partitionLocation = tableStorePath + File.separator + "partition" + File.separator +
-                              schemaName + File.separator + cubeName
+                              dbName + File.separator + tableName
       val partitionFileType = FileFactory.getFileType(partitionLocation)
       if (FileFactory.isFileExist(partitionLocation, partitionFileType)) {
         CarbonUtil
@@ -575,20 +535,20 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     }
 
     try {
-      sqlContext.sql(s"DROP TABLE $schemaName.$cubeName").collect()
+      sqlContext.sql(s"DROP TABLE $dbName.$tableName").collect()
     } catch {
       case e: Exception =>
         LOGGER.audit(
-          s"Error While deleting the table $schemaName.$cubeName during drop Table" + e.getMessage)
+          s"Error While deleting the table $dbName.$tableName during drop Table" + e.getMessage)
     }
 
     metadata.cubesMeta -= metadata.cubesMeta.filter(
-      c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(schemaName) &&
-           c.carbonTableIdentifier.getTableName.equalsIgnoreCase(cubeName))(0)
+      c => c.carbonTableIdentifier.getDatabaseName.equalsIgnoreCase(dbName) &&
+           c.carbonTableIdentifier.getTableName.equalsIgnoreCase(tableName))(0)
     org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
-      .removeTable(schemaName + "_" + cubeName)
-    logInfo(s"Table $cubeName of $schemaName Database dropped syccessfully.")
-    LOGGER.info("Table " + cubeName + " of " + schemaName + " Database dropped syccessfully.")
+      .removeTable(dbName + "_" + tableName)
+    logInfo(s"Table $tableName of $dbName Database dropped syccessfully.")
+    LOGGER.info("Table " + tableName + " of " + dbName + " Database dropped syccessfully.")
 
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala
index b1c3924..c01a937 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala
@@ -63,19 +63,7 @@ class CarbonRawStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan
               detailQuery = true,
               useBinaryAggregation = false)(sqlContext)._1 :: Nil
           }
-
-        case catalyst.planning.PartialAggregation(
-        namedGroupingAttributes,
-        rewrittenAggregateExpressions,
-        groupingExpressions,
-        partialComputation,
-        PhysicalOperation(projectList, predicates,
-        l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))) =>
-          handleRawAggregation(plan, plan, projectList, predicates, carbonRelation,
-            l, partialComputation, groupingExpressions, namedGroupingAttributes,
-            rewrittenAggregateExpressions)
-        case CarbonDictionaryCatalystDecoder(relations, profile,
-               aliasMap, _, child) =>
+        case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
           CarbonDictionaryDecoder(relations,
             profile,
             aliasMap,
@@ -85,47 +73,6 @@ class CarbonRawStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan
       }
     }
 
-
-    def handleRawAggregation(plan: LogicalPlan,
-        aggPlan: LogicalPlan,
-        projectList: Seq[NamedExpression],
-        predicates: Seq[Expression],
-        carbonRelation: CarbonDatasourceRelation,
-        logicalRelation: LogicalRelation,
-        partialComputation: Seq[NamedExpression],
-        groupingExpressions: Seq[Expression],
-        namedGroupingAttributes: Seq[Attribute],
-        rewrittenAggregateExpressions: Seq[NamedExpression]):
-    Seq[SparkPlan] = {
-      val groupByPresentOnMsr = isGroupByPresentOnMeasures(groupingExpressions,
-        carbonRelation.carbonRelation.metaData.carbonTable)
-      if(!groupByPresentOnMsr) {
-        val s = carbonRawScan(projectList,
-          predicates,
-          carbonRelation,
-          logicalRelation,
-          Some(partialComputation),
-          detailQuery = false,
-          useBinaryAggregation = true)(sqlContext)
-        // If any aggregate function present on dimnesions then don't use this plan.
-        if (!s._2) {
-          CarbonAggregate(
-            partial = false,
-            namedGroupingAttributes,
-            rewrittenAggregateExpressions,
-            CarbonRawAggregate(
-              partial = true,
-              groupingExpressions,
-              partialComputation,
-              s._1))(sqlContext) :: Nil
-        } else {
-          Nil
-        }
-      } else {
-        Nil
-      }
-    }
-
     /**
      * Create carbon scan
      */
@@ -141,13 +88,6 @@ class CarbonRawStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan
         relation.carbonRelation.metaData.carbonTable.getFactTableName.toLowerCase
       // Check out any expressions are there in project list. if they are present then we need to
       // decode them as well.
-      val projectExprsNeedToDecode = new java.util.HashSet[Attribute]()
-      projectList.map {
-        case attr: AttributeReference =>
-        case Alias(attr: AttributeReference, _) =>
-        case others =>
-          others.references.map(f => projectExprsNeedToDecode.add(f))
-      }
       val projectSet = AttributeSet(projectList.flatMap(_.references))
       val scan = CarbonRawTableScan(projectSet.toSeq,
         relation.carbonRelation,
@@ -155,13 +95,19 @@ class CarbonRawStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan
         groupExprs,
         useBinaryAggregation)(sqlContext)
       val dimAggrsPresence: Boolean = scan.buildCarbonPlan.getDimAggregatorInfos.size() > 0
-      projectExprsNeedToDecode.addAll(scan.attributesNeedToDecode)
+      projectList.map {
+        case attr: AttributeReference =>
+        case Alias(attr: AttributeReference, _) =>
+        case others =>
+          others.references
+            .map(f => scan.attributesNeedToDecode.add(f.asInstanceOf[AttributeReference]))
+      }
       if (!detailQuery) {
-        if (projectExprsNeedToDecode.size > 0) {
+        if (scan.attributesNeedToDecode.size > 0) {
           val decoder = getCarbonDecoder(logicalRelation,
             sc,
             tableName,
-            projectExprsNeedToDecode.asScala.toSeq,
+            scan.attributesNeedToDecode.asScala.toSeq,
             scan)
           if (scan.unprocessedExprs.nonEmpty) {
             val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
@@ -173,11 +119,11 @@ class CarbonRawStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan
           (scan, dimAggrsPresence)
         }
       } else {
-        if (projectExprsNeedToDecode.size() > 0) {
+        if (scan.attributesNeedToDecode.size() > 0) {
           val decoder = getCarbonDecoder(logicalRelation,
             sc,
             tableName,
-            projectExprsNeedToDecode.asScala.toSeq,
+            scan.attributesNeedToDecode.asScala.toSeq,
             scan)
           if (scan.unprocessedExprs.nonEmpty) {
             val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
index 6c7e362..56deb8a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
@@ -21,12 +21,15 @@ package org.apache.spark.sql.hive
 import scala.math.BigInt.int2bigInt
 
 import org.apache.spark.sql._
+import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
 import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation, QueryPlanner}
 import org.apache.spark.sql.catalyst.plans.Inner
 import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, Limit, LogicalPlan, Sort}
 import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, ExecutedCommand, Filter, Project, SparkPlan}
+import org.apache.spark.sql.execution.aggregate.Utils
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.datasources.{DescribeCommand => LogicalDescribeCommand, LogicalRelation}
 import org.apache.spark.sql.execution.joins.{BroadCastFilterPushJoin, BuildLeft, BuildRight}
@@ -75,44 +78,35 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
 
       case Limit(IntegerLiteral(limit),
       Sort(order, _,
-      p@PartialAggregation(namedGroupingAttributes,
-      rewrittenAggregateExpressions,
-      groupingExpressions,
-      partialComputation,
+      p@CarbonAggregation(groupingExpressions,
+      aggregateExpressions,
       PhysicalOperation(
       projectList,
       predicates,
       l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))))) =>
         val aggPlan = handleAggregation(plan, p, projectList, predicates, carbonRelation,
-          partialComputation, groupingExpressions, namedGroupingAttributes,
-          rewrittenAggregateExpressions)
+          groupingExpressions, aggregateExpressions)
         org.apache.spark.sql.execution.TakeOrderedAndProject(limit,
           order,
           None,
           aggPlan.head) :: Nil
 
-      case Limit(IntegerLiteral(limit), p@PartialAggregation(
-      namedGroupingAttributes,
-      rewrittenAggregateExpressions,
+      case Limit(IntegerLiteral(limit), p@CarbonAggregation(
       groupingExpressions,
-      partialComputation,
+      aggregateExpressions,
       PhysicalOperation(projectList, predicates,
       l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)))) =>
         val aggPlan = handleAggregation(plan, p, projectList, predicates, carbonRelation,
-          partialComputation, groupingExpressions, namedGroupingAttributes,
-          rewrittenAggregateExpressions)
+          groupingExpressions, aggregateExpressions)
         org.apache.spark.sql.execution.Limit(limit, aggPlan.head) :: Nil
 
-      case PartialAggregation(
-      namedGroupingAttributes,
-      rewrittenAggregateExpressions,
+      case CarbonAggregation(
       groupingExpressions,
-      partialComputation,
+      aggregateExpressions,
       PhysicalOperation(projectList, predicates,
       l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))) =>
         handleAggregation(plan, plan, projectList, predicates, carbonRelation,
-          partialComputation, groupingExpressions, namedGroupingAttributes,
-          rewrittenAggregateExpressions)
+          groupingExpressions, aggregateExpressions)
 
       case Limit(IntegerLiteral(limit),
       PhysicalOperation(projectList, predicates,
@@ -188,59 +182,138 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         projectList: Seq[NamedExpression],
         predicates: Seq[Expression],
         carbonRelation: CarbonDatasourceRelation,
-        partialComputation: Seq[NamedExpression],
         groupingExpressions: Seq[Expression],
-        namedGroupingAttributes: Seq[Attribute],
-        rewrittenAggregateExpressions: Seq[NamedExpression]):
+        namedGroupingAttributes: Seq[NamedExpression]):
     Seq[SparkPlan] = {
       val (_, _, _, _, groupExprs, substitutesortExprs, limitExpr) = extractPlan(plan)
 
       val s =
         try {
           carbonScan(projectList, predicates, carbonRelation.carbonRelation,
-            Some(partialComputation), substitutesortExprs, limitExpr, groupingExpressions.nonEmpty)
+            Some(namedGroupingAttributes), substitutesortExprs,
+            limitExpr, groupingExpressions.nonEmpty)
         } catch {
           case e: Exception => null
         }
 
       if (s != null) {
-        CarbonAggregate(
-          partial = false,
-          namedGroupingAttributes,
-          rewrittenAggregateExpressions,
-          CarbonAggregate(
-            partial = true,
-            groupingExpressions,
-            partialComputation,
-            s)(sqlContext))(sqlContext) :: Nil
+        aggregatePlan(groupingExpressions, namedGroupingAttributes, s)
 
       } else {
         (aggPlan, true) match {
-          case PartialAggregation(
-          namedGroupingAttributes,
-          rewrittenAggregateExpressions,
+          case CarbonAggregation(
           groupingExpressions,
-          partialComputation,
+          namedGroupingAttributes,
           PhysicalOperation(projectList, predicates,
           l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))) =>
             val (_, _, _, _, groupExprs, substitutesortExprs, limitExpr) = extractPlan(plan)
 
 
             val s = carbonScan(projectList, predicates, carbonRelation.carbonRelation,
-              Some(partialComputation), substitutesortExprs, limitExpr,
+              Some(namedGroupingAttributes), substitutesortExprs, limitExpr,
               groupingExpressions.nonEmpty, detailQuery = true)
 
-            CarbonAggregate(
-              partial = false,
-              namedGroupingAttributes,
-              rewrittenAggregateExpressions,
-              CarbonAggregate(
-                partial = true,
-                groupingExpressions,
-                partialComputation,
-                s)(sqlContext))(sqlContext) :: Nil
+            aggregatePlan(groupingExpressions, namedGroupingAttributes, s)
+        }
+      }
+    }
+
+    // TODO: It is duplicated code from spark. Need to find a way
+    private def aggregatePlan(groupingExpressions: Seq[Expression],
+        resultExpressions: Seq[NamedExpression],
+        child: SparkPlan) = {
+      // A single aggregate expression might appear multiple times in resultExpressions.
+      // In order to avoid evaluating an individual aggregate function multiple times, we'll
+      // build a set of the distinct aggregate expressions and build a function which can
+      // be used to re-write expressions so that they reference the single copy of the
+      // aggregate function which actually gets computed.
+      val aggregateExpressions = resultExpressions.flatMap { expr =>
+        expr.collect {
+          case agg: AggregateExpression => agg
         }
+      }.distinct
+      // For those distinct aggregate expressions, we create a map from the
+      // aggregate function to the corresponding attribute of the function.
+      val aggregateFunctionToAttribute = aggregateExpressions.map { agg =>
+        val aggregateFunction = agg.aggregateFunction
+        val attribute = Alias(aggregateFunction, aggregateFunction.toString)().toAttribute
+        (aggregateFunction, agg.isDistinct) -> attribute
+      }.toMap
+
+      val (functionsWithDistinct, functionsWithoutDistinct) =
+        aggregateExpressions.partition(_.isDistinct)
+      if (functionsWithDistinct.map(_.aggregateFunction.children).distinct.length > 1) {
+        // This is a sanity check. We should not reach here when we have multiple distinct
+        // column sets. Our MultipleDistinctRewriter should take care this case.
+        sys.error("You hit a query analyzer bug. Please report your query to " +
+                  "Spark user mailing list.")
       }
+
+      val namedGroupingExpressions = groupingExpressions.map {
+        case ne: NamedExpression => ne -> ne
+        // If the expression is not a NamedExpressions, we add an alias.
+        // So, when we generate the result of the operator, the Aggregate Operator
+        // can directly get the Seq of attributes representing the grouping expressions.
+        case other =>
+          val withAlias = Alias(other, other.toString)()
+          other -> withAlias
+      }
+      val groupExpressionMap = namedGroupingExpressions.toMap
+
+      // The original `resultExpressions` are a set of expressions which may reference
+      // aggregate expressions, grouping column values, and constants. When aggregate operator
+      // emits output rows, we will use `resultExpressions` to generate an output projection
+      // which takes the grouping columns and final aggregate result buffer as input.
+      // Thus, we must re-write the result expressions so that their attributes match up with
+      // the attributes of the final result projection's input row:
+      val rewrittenResultExpressions = resultExpressions.map { expr =>
+        expr.transformDown {
+          case AggregateExpression(aggregateFunction, _, isDistinct) =>
+            // The final aggregation buffer's attributes will be `finalAggregationAttributes`,
+            // so replace each aggregate expression by its corresponding attribute in the set:
+            aggregateFunctionToAttribute(aggregateFunction, isDistinct)
+          case expression =>
+            // Since we're using `namedGroupingAttributes` to extract the grouping key
+            // columns, we need to replace grouping key expressions with their corresponding
+            // attributes. We do not rely on the equality check at here since attributes may
+            // differ cosmetically. Instead, we use semanticEquals.
+            groupExpressionMap.collectFirst {
+              case (expr, ne) if expr semanticEquals expression => ne.toAttribute
+            }.getOrElse(expression)
+        }.asInstanceOf[NamedExpression]
+      }
+
+      val aggregateOperator =
+        if (aggregateExpressions.map(_.aggregateFunction).exists(!_.supportsPartial)) {
+          if (functionsWithDistinct.nonEmpty) {
+            sys.error("Distinct columns cannot exist in Aggregate operator containing " +
+                      "aggregate functions which don't support partial aggregation.")
+          } else {
+            Utils.planAggregateWithoutPartial(
+              namedGroupingExpressions.map(_._2),
+              aggregateExpressions,
+              aggregateFunctionToAttribute,
+              rewrittenResultExpressions,
+              child)
+          }
+        } else if (functionsWithDistinct.isEmpty) {
+          Utils.planAggregateWithoutDistinct(
+            namedGroupingExpressions.map(_._2),
+            aggregateExpressions,
+            aggregateFunctionToAttribute,
+            rewrittenResultExpressions,
+            child)
+        } else {
+          Utils.planAggregateWithOneDistinct(
+            namedGroupingExpressions.map(_._2),
+            functionsWithDistinct,
+            functionsWithoutDistinct,
+            aggregateFunctionToAttribute,
+            rewrittenResultExpressions,
+            child)
+        }
+
+      aggregateOperator
     }
 
     private def canPushDownJoin(otherRDDPlan: LogicalPlan,
@@ -333,8 +406,8 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         ExecutedCommand(ShowAllTablesDetail(schemaName, plan.output)) :: Nil
       case DropTable(tableName, ifNotExists)
         if CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .cubeExists(Seq(tableName.toLowerCase()))(sqlContext) =>
-        ExecutedCommand(DropCubeCommand(ifNotExists, None, tableName.toLowerCase())) :: Nil
+          .tableExists(TableIdentifier(tableName.toLowerCase))(sqlContext) =>
+        ExecutedCommand(DropCubeCommand(ifNotExists, None, tableName.toLowerCase)) :: Nil
       case ShowAggregateTablesCommand(schemaName) =>
         ExecutedCommand(ShowAggregateTables(schemaName, plan.output)) :: Nil
       case ShowLoadsCommand(schemaName, cube, limit) =>
@@ -342,7 +415,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
       case LoadCube(schemaNameOp, cubeName, factPathFromUser, dimFilesPath,
       partionValues, isOverwriteExist, inputSqlString) =>
         val isCarbonTable = CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .cubeExists(schemaNameOp, cubeName)(sqlContext)
+          .tableExists(TableIdentifier(cubeName, schemaNameOp))(sqlContext)
         if (isCarbonTable || partionValues.nonEmpty) {
           ExecutedCommand(LoadCube(schemaNameOp, cubeName, factPathFromUser,
             dimFilesPath, partionValues, isOverwriteExist, inputSqlString)) :: Nil
@@ -360,7 +433,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         }
       case DescribeFormattedCommand(sql, tblIdentifier) =>
         val isCube = CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .cubeExists(tblIdentifier)(sqlContext)
+          .tableExists(tblIdentifier)(sqlContext)
         if (isCube) {
           val describe =
             LogicalDescribeCommand(UnresolvedRelation(tblIdentifier, None), isExtended = false)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala
index c4c214d..58c02ff 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala
@@ -34,7 +34,7 @@ private[sql] object CarbonStrategy {
   }
 }
 
-private[spark] class CarbonSQLDialect extends HiveQLDialect {
+private[spark] class CarbonSQLDialect(context: HiveContext) extends HiveQLDialect(context) {
 
   @transient
   protected val sqlParser = new CarbonSqlParser

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
index a975317..d80c065 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
@@ -24,7 +24,8 @@ import scala.collection.mutable.ArrayBuffer
 
 import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.CatalystConf
-import org.apache.spark.sql.catalyst.expressions.{AggregateExpression, Attribute, _}
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.catalyst.optimizer.Optimizer
 import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, _}
 import org.apache.spark.sql.catalyst.rules.Rule
@@ -122,7 +123,7 @@ class CarbonOptimizer(optimizer: Optimizer, conf: CatalystConf)
             }
             var child = agg.child
             // Incase if the child also aggregate then push down decoder to child
-            if (attrsOndimAggs.size() > 0 && !child.isInstanceOf[Aggregate]) {
+            if (attrsOndimAggs.size() > 0 && !(child.equals(agg))) {
               child = CarbonDictionaryTempDecoder(attrsOndimAggs,
                 new util.HashSet[Attribute](),
                 agg.child)
@@ -398,10 +399,10 @@ class CarbonOptimizer(optimizer: Optimizer, conf: CatalystConf)
         allAttrsNotDecode: util.Set[Attribute],
         aliasMap: CarbonAliasDecoderRelation) = {
       val uAttr = aliasMap.getOrElse(attr, attr)
-      val relation = relations.find(p => p.attributeMap.contains(uAttr))
+      val relation = relations.find(p => p.contains(uAttr))
       if (relation.isDefined) {
         relation.get.carbonRelation.carbonRelation.metaData.dictionaryMap.get(uAttr.name) match {
-          case Some(true) if !allAttrsNotDecode.contains(uAttr) =>
+          case Some(true) if !allAttrsNotDecode.asScala.exists(p => p.name.equals(uAttr.name)) =>
             val newAttr = AttributeReference(attr.name,
               IntegerType,
               attr.nullable,
@@ -419,7 +420,7 @@ class CarbonOptimizer(optimizer: Optimizer, conf: CatalystConf)
         relations: Seq[CarbonDecoderRelation],
         aliasMap: CarbonAliasDecoderRelation): Boolean = {
       val uAttr = aliasMap.getOrElse(attr, attr)
-      val relation = relations.find(p => p.attributeMap.contains(uAttr))
+      val relation = relations.find(p => p.contains(uAttr))
       if (relation.isDefined) {
         relation.get.carbonRelation.carbonRelation.metaData.dictionaryMap.get(uAttr.name) match {
           case Some(true) => true
@@ -459,9 +460,16 @@ case class CarbonDecoderRelation(
   }
 
   def contains(attr: Attribute): Boolean = {
-    attributeMap
-       .exists(entry => entry._1.name.equals(attr.name) && entry._1.exprId.equals(attr.exprId)) ||
-     extraAttrs.exists(entry => entry.name.equals(attr.name) && entry.exprId.equals(attr.exprId))
+    var exists =
+      attributeMap.exists(entry => entry._1.name.equalsIgnoreCase(attr.name) &&
+                        entry._1.exprId.equals(attr.exprId)) ||
+      extraAttrs.exists(entry => entry.name.equalsIgnoreCase(attr.name) &&
+                                entry.exprId.equals(attr.exprId))
+    if(!exists) {
+      exists = attributeMap.exists(entry => entry._1.name.equalsIgnoreCase(attr.name)) ||
+        extraAttrs.exists(entry => entry.name.equalsIgnoreCase(attr.name) )
+    }
+    exists
   }
 }
 
@@ -474,7 +482,8 @@ case class CarbonAliasDecoderRelation() {
   }
 
   def getOrElse(key: Attribute, default: Attribute): Attribute = {
-    val value = attrMap.find(p => p._1.name.equals(key.name) && p._1.exprId.equals(key.exprId))
+    val value = attrMap.find(p =>
+      p._1.name.equalsIgnoreCase(key.name) && p._1.exprId.equals(key.exprId))
     value match {
       case Some((k, v)) => v
       case _ => default

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
index 8c6df4c..aee375a 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
@@ -27,6 +27,9 @@ import org.apache.spark.sql.catalyst.expressions._
 import org.apache.spark.sql.optimizer.CarbonAliasDecoderRelation
 import org.apache.spark.sql.types.StructType
 
+import org.carbondata.core.carbon.metadata.datatype.DataType
+import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn
 import org.carbondata.query.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
 import org.carbondata.query.expression.conditional._
 import org.carbondata.query.expression.logical.{AndExpression, OrExpression}
@@ -160,7 +163,8 @@ object CarbonFilters {
 
   def processExpression(exprs: Seq[Expression],
       attributesNeedToDecode: java.util.HashSet[AttributeReference],
-      unprocessedExprs: ArrayBuffer[Expression]): Option[CarbonExpression] = {
+      unprocessedExprs: ArrayBuffer[Expression],
+      carbonTable: CarbonTable): Option[CarbonExpression] = {
     def transformExpression(expr: Expression): Option[CarbonExpression] = {
       expr match {
         case Or(left, right) =>
@@ -208,8 +212,9 @@ object CarbonFilters {
             new ListExpression(list.map(transformExpression(_).get).asJava)))
 
         case AttributeReference(name, dataType, _, _) =>
-          Some(new CarbonColumnExpression(name.toString,
-            CarbonScalaUtil.convertSparkToCarbonDataType(dataType)))
+          Some(new CarbonColumnExpression(name,
+            CarbonScalaUtil.convertSparkToCarbonDataType(
+              getActualCarbonDataType(name, carbonTable))))
         case FakeCarbonCast(literal, dataType) => transformExpression(literal)
         case Literal(name, dataType) => Some(new
             CarbonLiteralExpression(name, CarbonScalaUtil.convertSparkToCarbonDataType(dataType)))
@@ -225,4 +230,19 @@ object CarbonFilters {
     exprs.flatMap(transformExpression).reduceOption(new AndExpression(_, _))
   }
 
+  private def getActualCarbonDataType(column: String, carbonTable: CarbonTable) = {
+    var carbonColumn: CarbonColumn =
+      carbonTable.getDimensionByName(carbonTable.getFactTableName, column)
+    val dataType = if (carbonColumn != null) {
+      carbonColumn.getDataType
+    } else {
+      carbonColumn = carbonTable.getMeasureByName(carbonTable.getFactTableName, column)
+      carbonColumn.getDataType match {
+        case DataType.LONG => DataType.LONG
+        case DataType.DECIMAL => DataType.DECIMAL
+        case _ => DataType.DOUBLE
+      }
+    }
+    CarbonScalaUtil.convertCarbonToSparkDataType(dataType)
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/CarbonOption.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/CarbonOption.scala b/integration/spark/src/main/scala/org/carbondata/spark/CarbonOption.scala
index b91cf98..12f3dc4 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/CarbonOption.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/CarbonOption.scala
@@ -21,7 +21,7 @@ package org.carbondata.spark
  * Contains all options for Spark data source
  */
 class CarbonOption(options: Map[String, String]) {
-  def tableIdentifier: String = options.getOrElse("cubeName", s"$dbName.$tableName")
+  def tableIdentifier: String = options.getOrElse("tableName", s"$dbName.$tableName")
 
   def dbName: String = options.getOrElse("dbName", "default")
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
index d906745..6ee882b 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
@@ -47,6 +47,14 @@ class RawKeyValImpl extends RawKeyVal[BatchRawResult, Any] {
   override def getKey(key: BatchRawResult, value: Any): (BatchRawResult, Any) = (key, value)
 }
 
+trait RawKey[K, V] extends Serializable {
+  def getKey(key: Array[Any], value: Any): (K, V)
+
+}
+
+class RawKeyImpl extends RawKey[Array[Any], Any] {
+  override def getKey(key: Array[Any], value: Any): (Array[Any], Any) = (key, value)
+}
 trait Result[K, V] extends Serializable {
   def getKey(key: Int, value: LoadMetadataDetails): (K, V)
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala b/integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
deleted file mode 100644
index 65e7538..0000000
--- a/integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
+++ /dev/null
@@ -1,807 +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.carbondata.spark.agg
-
-import scala.language.implicitConversions
-
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
-import org.apache.spark.sql.types._
-
-import org.carbondata.query.aggregator.MeasureAggregator
-import org.carbondata.query.aggregator.impl._
-
-case class CountCarbon(child: Expression) extends UnaryExpression with PartialAggregate1 {
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"COUNT($child)"
-
-  override def asPartial: SplitEvaluation = {
-    val partialCount = Alias(CountCarbon(child), "PartialCount")()
-    SplitEvaluation(CountCarbonFinal(partialCount.toAttribute, LongType), partialCount :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = new CountFunctionCarbon(child, this, false)
-
-  implicit def toAggregates(aggregate: MeasureAggregator): Double = aggregate.getDoubleValue()
-}
-
-case class CountCarbonFinal(child: Expression, origDataType: DataType)
-  extends AggregateExpression1 {
-
-  override def children: Seq[Expression] = child :: Nil
-
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = origDataType
-
-  override def toString: String = s"COUNT($child)"
-
-  override def newInstance(): AggregateFunction1 = new CountFunctionCarbon(child, this, true)
-}
-
-
-case class CountDistinctCarbon(child: Expression) extends PartialAggregate1 {
-  override def children: Seq[Expression] = child :: Nil
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"COUNT(DISTINCT ($child)"
-
-  override def asPartial: SplitEvaluation = {
-    val partialSet = Alias(CountDistinctCarbon(child), "partialSets")()
-    SplitEvaluation(
-      CountDistinctCarbonFinal(partialSet.toAttribute, LongType),
-      partialSet :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = new CountDistinctFunctionCarbon(child, this)
-}
-
-case class CountDistinctCarbonFinal(inputSet: Expression, origDataType: DataType)
-  extends AggregateExpression1 {
-  override def children: Seq[Expression] = inputSet :: Nil
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = origDataType
-
-  override def toString: String = s"COUNTFINAL(DISTINCT ${ inputSet }})"
-
-  override def newInstance(): AggregateFunction1 = {
-    new CountDistinctFunctionCarbonFinal(inputSet, this)
-  }
-}
-
-case class AverageCarbon(child: Expression, castedDataType: DataType = null)
-  extends UnaryExpression with PartialAggregate1 {
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"AVGCarbon($child)"
-
-  override def asPartial: SplitEvaluation = {
-    val partialSum = Alias(AverageCarbon(child), "PartialAverage")()
-    SplitEvaluation(
-      AverageCarbonFinal(partialSum.toAttribute,
-        child.dataType match {
-          case IntegerType | StringType | LongType | TimestampType => DoubleType
-          case _ => child.dataType
-        }),
-      partialSum :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = new AverageFunctionCarbon(child, this, false)
-}
-
-case class AverageCarbonFinal(child: Expression, origDataType: DataType)
-  extends AggregateExpression1 {
-  override def children: Seq[Expression] = child :: Nil
-
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = origDataType
-
-  override def toString: String = s"AVG($child)"
-
-  override def newInstance(): AggregateFunction1 = new AverageFunctionCarbon(child, this, true)
-}
-
-case class SumCarbon(child: Expression, castedDataType: DataType = null)
-  extends UnaryExpression with PartialAggregate1 {
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"SUMCarbon($child)"
-
-  override def asPartial: SplitEvaluation = {
-    val partialSum = Alias(SumCarbon(child), "PartialSum")()
-    SplitEvaluation(
-      SumCarbonFinal(partialSum.toAttribute,
-        if (castedDataType != null) castedDataType else child.dataType),
-      partialSum :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = new SumFunctionCarbon(child, this, false)
-
-  implicit def toAggregates(aggregate: MeasureAggregator): Double = aggregate.getDoubleValue()
-}
-
-case class SumCarbonFinal(child: Expression, origDataType: DataType) extends AggregateExpression1 {
-  override def children: Seq[Expression] = child :: Nil
-
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = origDataType
-
-  override def toString: String = s"SUMCarbonFinal($child)"
-
-  override def newInstance(): AggregateFunction1 = new SumFunctionCarbon(child, this, true)
-}
-
-case class MaxCarbon(child: Expression, castedDataType: DataType = null)
-  extends UnaryExpression with PartialAggregate1 {
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"MaxCarbon($child)"
-
-  // to do partialSum to PartialMax many places
-  override def asPartial: SplitEvaluation = {
-    val partialSum = Alias(MaxCarbon(child), "PartialMax")()
-    SplitEvaluation(
-      MaxCarbonFinal(partialSum.toAttribute,
-        if (castedDataType != null) castedDataType else child.dataType),
-      partialSum :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = new MaxFunctionCarbon(child, this, false)
-}
-
-case class MaxCarbonFinal(child: Expression, origDataType: DataType) extends AggregateExpression1 {
-  override def children: Seq[Expression] = child :: Nil
-
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = origDataType
-
-  override def toString: String = s"MaxCarbonFinal($child)"
-
-  override def newInstance(): AggregateFunction1 = new MaxFunctionCarbon(child, this, true)
-}
-
-case class MinCarbon(child: Expression, castedDataType: DataType = null)
-  extends UnaryExpression with PartialAggregate1 {
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"MinCarbon($child)"
-
-  override def asPartial: SplitEvaluation = {
-    val partialSum = Alias(MinCarbon(child), "PartialMin")()
-    SplitEvaluation(
-      MinCarbonFinal(partialSum.toAttribute,
-        if (castedDataType != null) castedDataType else child.dataType),
-      partialSum :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = new MinFunctionCarbon(child, this, false)
-}
-
-case class MinCarbonFinal(child: Expression, origDataType: DataType) extends AggregateExpression1 {
-  override def children: Seq[Expression] = child :: Nil
-
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = origDataType
-
-  override def toString: String = s"MinCarbonFinal($child)"
-
-  override def newInstance(): AggregateFunction1 = new MinFunctionCarbon(child, this, true)
-}
-
-case class SumDistinctCarbon(child: Expression, castedDataType: DataType = null)
-  extends UnaryExpression with PartialAggregate1 {
-
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"PARTIAL_SUM_DISTINCT($child)"
-
-  override def asPartial: SplitEvaluation = {
-    val partialSum = Alias(SumDistinctCarbon(child), "PartialSumDistinct")()
-    SplitEvaluation(
-      SumDistinctFinalCarbon(partialSum.toAttribute,
-        if (castedDataType != null) {
-          castedDataType
-        } else {
-          child.dataType
-        }),
-      partialSum :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = {
-    new SumDisctinctFunctionCarbon(child, this, false)
-  }
-}
-
-case class SumDistinctFinalCarbon(child: Expression, origDataType: DataType)
-  extends AggregateExpression1 {
-  override def children: Seq[Expression] = child :: Nil
-
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = false
-
-  override def dataType: DataType = origDataType
-
-  override def toString: String = s"FINAL_SUM_DISTINCT($child)"
-
-  override def newInstance(): AggregateFunction1 = new SumDisctinctFunctionCarbon(child, this, true)
-}
-
-case class FirstCarbon(child: Expression, origDataType: DataType = MeasureAggregatorUDT)
-  extends UnaryExpression with PartialAggregate1 {
-  override def references: AttributeSet = child.references
-
-  override def nullable: Boolean = child.nullable
-
-  override def dataType: DataType = MeasureAggregatorUDT
-
-  override def toString: String = s"FIRST($child)"
-
-  override def asPartial: SplitEvaluation = {
-    val partialFirst = Alias(FirstCarbon(child), "PartialFirst")()
-    SplitEvaluation(
-      FirstCarbon(partialFirst.toAttribute, child.dataType),
-      partialFirst :: Nil)
-  }
-
-  override def newInstance(): AggregateFunction1 = new FirstFunctionCarbon(child, this)
-}
-
-case class AverageFunctionCarbon(expr: Expression, base: AggregateExpression1, finalAgg: Boolean)
-  extends AggregateFunction1 {
-
-  def this() = this(null, null, false) // Required for serialization.
-
-  //  var count: Int = _
-  private var avg: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case s: MeasureAggregator => s
-      case s =>
-        var dc: MeasureAggregator = null
-        if (s != null) {
-          s match {
-            case v: java.math.BigDecimal =>
-              dc = new AvgBigDecimalAggregator
-              dc.agg(new java.math.BigDecimal(s.toString))
-              dc.setNewValue(new java.math.BigDecimal(s.toString))
-            case l: Long =>
-              dc = new AvgLongAggregator
-              dc.agg(s.toString.toLong)
-              dc.setNewValue(s.toString.toLong)
-            case _ =>
-              dc = new AvgDoubleAggregator
-              dc.agg(s.toString.toDouble)
-              dc.setNewValue(s.toString.toDouble)
-          }
-        }
-        else {
-          dc = new AvgDoubleAggregator()
-        }
-        dc
-    }
-    if (avg == null) {
-      avg = agg
-    } else {
-      avg.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any = {
-    if (finalAgg) {
-      if (avg.isFirstTime) {
-        null
-      } else {
-        avg match {
-          case avg: AvgBigDecimalAggregator =>
-            Cast(Literal(avg.getBigDecimalValue), base.dataType).eval(null)
-          case avg: AvgLongAggregator =>
-            Cast(Literal(avg.getLongValue), base.dataType).eval(null)
-          case _ =>
-            Cast(Literal(avg.getDoubleValue), base.dataType).eval(null)
-        }
-      }
-    } else {
-      avg
-    }
-  }
-}
-
-case class CountFunctionCarbon(expr: Expression, base: AggregateExpression1, finalAgg: Boolean)
-  extends AggregateFunction1 {
-  def this() = this(null, null, false) // Required for serialization.
-
-  private var count: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case m: MeasureAggregator => m
-      case others =>
-        val agg1: MeasureAggregator = new CountAggregator
-        if (others != null) {
-          agg1.agg(0)
-        }
-        agg1
-    }
-    if (count == null) {
-      count = agg
-    } else {
-      count.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any = {
-    if (finalAgg && count != null) {
-      if (count.isFirstTime) {
-        0L
-      } else {
-        Cast(Literal(count.getDoubleValue), base.dataType).eval(null)
-      }
-    } else {
-      count
-    }
-  }
-
-}
-
-
-case class SumFunctionCarbon(expr: Expression, base: AggregateExpression1, finalAgg: Boolean)
-  extends AggregateFunction1 {
-  def this() = this(null, null, false) // Required for serialization.
-
-  private var sum: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case s: MeasureAggregator => s
-      case s =>
-        var dc: MeasureAggregator = null
-        if (s != null) {
-          s match {
-            case bd: java.math.BigDecimal =>
-              dc = new SumBigDecimalAggregator
-              dc.agg(new java.math.BigDecimal(s.toString))
-              dc.setNewValue(new java.math.BigDecimal(s.toString))
-            case l: Long =>
-              dc = new SumLongAggregator
-              dc.agg(s.toString.toLong)
-              dc.setNewValue(s.toString.toLong)
-            case _ =>
-              dc = new SumDoubleAggregator
-              dc.agg(s.toString.toDouble)
-              dc.setNewValue(s.toString.toDouble)
-          }
-        }
-        else {
-          dc = new SumDoubleAggregator
-        }
-        dc
-    }
-    if (sum == null) {
-      sum = agg
-    } else {
-      sum.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any = {
-    if (finalAgg && sum != null) {
-      if (sum.isFirstTime) {
-        null
-      } else {
-        sum match {
-          case s: SumBigDecimalAggregator =>
-            Cast(Literal(sum.getBigDecimalValue), base.dataType).eval(input)
-          case s: SumLongAggregator =>
-            Cast(Literal(sum.getLongValue), base.dataType).eval(input)
-          case _ =>
-            Cast(Literal(sum.getDoubleValue), base.dataType).eval(input)
-        }
-      }
-    } else {
-      sum
-    }
-  }
-}
-
-case class MaxFunctionCarbon(expr: Expression, base: AggregateExpression1, finalAgg: Boolean)
-  extends AggregateFunction1 {
-  def this() = this(null, null, false) // Required for serialization.
-
-  private var max: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case s: MeasureAggregator => s
-      case s =>
-        val dc = new MaxAggregator
-        if (s != null) {
-          dc.agg(s.toString.toDouble)
-          dc.setNewValue(s.toString.toDouble)
-        }
-        dc
-    }
-    if (max == null) {
-      max = agg
-    } else {
-      max.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any = {
-    if (finalAgg && max != null) {
-      if (max.isFirstTime) {
-        null
-      } else {
-        Cast(Literal(max.getValueObject), base.dataType).eval(null)
-      }
-    } else {
-      max
-    }
-  }
-}
-
-case class MinFunctionCarbon(expr: Expression, base: AggregateExpression1, finalAgg: Boolean)
-  extends AggregateFunction1 {
-  def this() = this(null, null, false) // Required for serialization.
-
-  private var min: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case s: MeasureAggregator => s
-      case s =>
-        val dc: MeasureAggregator = new MinAggregator
-        if (s != null) {
-          dc.agg(s.toString.toDouble)
-          dc.setNewValue(s.toString.toDouble)
-        }
-        dc
-    }
-    if (min == null) {
-      min = agg
-    } else {
-      min.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any = {
-    if (finalAgg && min != null) {
-      if (min.isFirstTime) {
-        null
-      } else {
-        Cast(Literal(min.getValueObject), base.dataType).eval(null)
-      }
-    } else {
-      min
-    }
-  }
-}
-
-case class SumDisctinctFunctionCarbon(expr: Expression, base: AggregateExpression1,
-    isFinal: Boolean)
-  extends AggregateFunction1 {
-
-  def this() = this(null, null, false) // Required for serialization.
-
-  private var distinct: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case s: MeasureAggregator => s
-      case null => null
-      case s =>
-        var dc: MeasureAggregator = null
-        s match {
-          case Double =>
-            dc = new SumDistinctDoubleAggregator
-            dc.setNewValue(s.toString.toDouble)
-          case Int =>
-            dc = new SumDistinctLongAggregator
-            dc.setNewValue(s.toString.toLong)
-          case bd: java.math.BigDecimal =>
-            dc = new SumDistinctBigDecimalAggregator
-            dc.setNewValue(new java.math.BigDecimal(s.toString))
-          case _ =>
-        }
-        dc
-    }
-    if (distinct == null) {
-      distinct = agg
-    } else {
-      distinct.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any =
-  // in case of empty load it was failing so added null check.
-  {
-    if (isFinal && distinct != null) {
-      if (distinct.isFirstTime) {
-        null
-      }
-      else {
-      Cast(Literal(distinct.getValueObject), base.dataType).eval(null)
-      }
-    }
-    else {
-      distinct
-    }
-  }
-}
-
-case class CountDistinctFunctionCarbon(expr: Expression, base: AggregateExpression1)
-  extends AggregateFunction1 {
-
-  def this() = this(null, null) // Required for serialization.
-
-  private var count: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case s: MeasureAggregator => s
-      case null => null
-      case s =>
-        val dc = new DistinctCountAggregatorObjectSet
-        dc.setNewValue(s.toString)
-        dc
-    }
-    if (count == null) {
-      count = agg
-    } else {
-      count.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any = count
-}
-
-case class CountDistinctFunctionCarbonFinal(expr: Expression, base: AggregateExpression1)
-  extends AggregateFunction1 {
-
-  def this() = this(null, null) // Required for serialization.
-
-  private var count: MeasureAggregator = null
-
-  override def update(input: InternalRow): Unit = {
-    val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-    val resolution =
-      if (br.isDefined) {
-        input.get(br.get.ordinal, MeasureAggregatorUDT)
-      } else {
-        expr.eval(input)
-      }
-    val agg = resolution match {
-      case s: MeasureAggregator => s
-      case null => null
-      case s =>
-        val dc = new DistinctCountAggregatorObjectSet
-        dc.setNewValue(s.toString)
-        dc
-    }
-    if (count == null) {
-      count = agg
-    } else {
-      count.merge(agg)
-    }
-  }
-
-  override def eval(input: InternalRow): Any = {
-    if (count == null) {
-      Cast(Literal(0), base.dataType).eval(null)
-    } else if (count.isFirstTime) {
-      Cast(Literal(0), base.dataType).eval(null)
-    } else {
-      Cast(Literal(count.getDoubleValue), base.dataType).eval(null)
-    }
-  }
-}
-
-case class FirstFunctionCarbon(expr: Expression, base: AggregateExpression1)
-  extends AggregateFunction1 {
-  def this() = this(null, null) // Required for serialization.
-
-  var result: Any = null
-
-  override def update(input: InternalRow): Unit = {
-    if (result == null) {
-      val br = expr.collectFirst({ case a@BoundReference(_, _, _) => a })
-      val resolution =
-        if (br.isDefined) {
-          input.get(br.get.ordinal, MeasureAggregatorUDT)
-        } else {
-          expr.eval(input)
-        }
-
-      result = resolution
-    }
-  }
-
-  override def eval(input: InternalRow): Any = Cast(Literal(result), base.dataType).eval(null)
-}
-
-case class FlattenExpr(expr: Expression) extends Expression with CodegenFallback {
-  self: Product =>
-
-  override def children: Seq[Expression] = Seq(expr)
-
-  override def dataType: DataType = expr.dataType
-
-  override def nullable: Boolean = expr.nullable
-
-  override def references: AttributeSet = AttributeSet(expr.flatMap(_.references.iterator))
-
-  override def foldable: Boolean = expr.foldable
-
-  override def toString: String = "Flatten(" + expr.toString + ")"
-
-  type EvaluatedType = Any
-
-  override def eval(input: InternalRow): Any = {
-    expr.eval(input) match {
-      case d: MeasureAggregator => d.getDoubleValue
-      case others => others
-    }
-  }
-}
-
-case class FlatAggregatorsExpr(expr: Expression) extends Expression with CodegenFallback {
-  self: Product =>
-
-  override def children: Seq[Expression] = Seq(expr)
-
-  override def dataType: DataType = expr.dataType
-
-  override def nullable: Boolean = expr.nullable
-
-  override def references: AttributeSet = AttributeSet(expr.flatMap(_.references.iterator))
-
-  override def foldable: Boolean = expr.foldable
-
-  override def toString: String = "FlattenAggregators(" + expr.toString + ")"
-
-  type EvaluatedType = Any
-
-  override def eval(input: InternalRow): Any = {
-    expr.eval(input) match {
-      case d: MeasureAggregator =>
-        d.setNewValue(d.getDoubleValue)
-        d
-      case others => others
-    }
-  }
-}
-
-case class PositionLiteral(expr: Expression, intermediateDataType: DataType)
-  extends LeafExpression with CodegenFallback {
-  override def dataType: DataType = expr.dataType
-
-  override def nullable: Boolean = false
-
-  type EvaluatedType = Any
-  var position = -1
-
-  def setPosition(pos: Int): Unit = position = pos
-
-  override def toString: String = s"PositionLiteral($position : $expr)"
-
-  override def eval(input: InternalRow): Any = {
-    if (position != -1) {
-      input.get(position, intermediateDataType)
-    } else {
-      expr.eval(input)
-    }
-  }
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/agg/MeasureAggregatorUDT.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/agg/MeasureAggregatorUDT.scala b/integration/spark/src/main/scala/org/carbondata/spark/agg/MeasureAggregatorUDT.scala
deleted file mode 100644
index ea0c8af..0000000
--- a/integration/spark/src/main/scala/org/carbondata/spark/agg/MeasureAggregatorUDT.scala
+++ /dev/null
@@ -1,55 +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.carbondata.spark.agg
-
-import org.apache.spark.sql.types._
-
-import org.carbondata.query.aggregator.MeasureAggregator
-
-/**
- * class to support user defined type for carbon measure aggregators
- * from spark 1.5, spark has made the data type strict and ANY is no more supported
- * for every data, we need to give the data type
- */
-class MeasureAggregatorUDT extends UserDefinedType[MeasureAggregator] {
-  // the default DoubleType is Ok as we are not going to pass to spark sql to
-  // evaluate,need to add this for compilation errors
-  override def sqlType: DataType = {
-    ArrayType(DoubleType, containsNull = false)
-  }
-
-  override def serialize(obj: Any): Any = {
-    obj match {
-      case p: MeasureAggregator => p
-    }
-  }
-
-  override def deserialize(datum: Any): MeasureAggregator = {
-    datum match {
-      case values =>
-        val xy = values.asInstanceOf[MeasureAggregator]
-        xy
-    }
-  }
-
-  override def userClass: Class[MeasureAggregator] = classOf[MeasureAggregator]
-
-  override def asNullable: MeasureAggregatorUDT = this
-}
-
-case object MeasureAggregatorUDT extends MeasureAggregatorUDT

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index ded665e..0672281 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -22,7 +22,7 @@ import java.util
 import java.util.concurrent.{Executors, ExecutorService}
 
 import scala.collection.JavaConverters._
-import scala.collection.mutable.{ArrayBuffer, ListBuffer}
+import scala.collection.mutable.ListBuffer
 import scala.util.control.Breaks._
 
 import org.apache.hadoop.conf.{Configurable, Configuration}
@@ -189,9 +189,7 @@ object CarbonDataRDDFactory extends Logging {
         if (carbonLock.lockWithRetries()) {
           logInfo("Successfully got the table metadata file lock")
           if (updatedLoadMetadataDetailsList.nonEmpty) {
-            LoadAggregateTabAfterRetention(schemaName, cube.getFactTableName, cube.getFactTableName,
-              sqlContext, schema, updatedLoadMetadataDetailsList
-            )
+            // TODO: Load Aggregate tables after retention.
           }
 
           // write
@@ -217,56 +215,6 @@ object CarbonDataRDDFactory extends Logging {
     }
   }
 
-  def LoadAggregateTabAfterRetention(
-      schemaName: String,
-      cubeName: String,
-      factTableName: String,
-      sqlContext: SQLContext,
-      schema: CarbonDataLoadSchema,
-      list: ListBuffer[LoadMetadataDetails]) {
-    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
-      Option(schemaName),
-      cubeName,
-      None
-    )(sqlContext).asInstanceOf[CarbonRelation]
-    if (relation == null) {
-      sys.error(s"Table $schemaName.$cubeName does not exist")
-    }
-    val carbonLoadModel = new CarbonLoadModel()
-    carbonLoadModel.setTableName(cubeName)
-    carbonLoadModel.setDatabaseName(schemaName)
-    val table = relation.cubeMeta.carbonTable
-    val aggTables = schema.getCarbonTable.getAggregateTablesName
-    if (null != aggTables && !aggTables.isEmpty) {
-      carbonLoadModel.setRetentionRequest(true)
-      carbonLoadModel.setLoadMetadataDetails(list.asJava)
-      carbonLoadModel.setTableName(table.getFactTableName)
-      carbonLoadModel
-        .setCarbonDataLoadSchema(new CarbonDataLoadSchema(relation.cubeMeta.carbonTable))
-      // TODO: need to fill dimension relation from data load sql command
-      var storeLocation = CarbonProperties.getInstance
-        .getProperty(CarbonCommonConstants.STORE_LOCATION_TEMP_PATH,
-          System.getProperty("java.io.tmpdir")
-        )
-      storeLocation = storeLocation + "/carbonstore/" + System.currentTimeMillis()
-      val columinar = sqlContext.getConf("carbon.is.columnar.storage", "true").toBoolean
-      var kettleHomePath = sqlContext.getConf("carbon.kettle.home", null)
-      if (null == kettleHomePath) {
-        kettleHomePath = CarbonProperties.getInstance.getProperty("carbon.kettle.home")
-      }
-      if (kettleHomePath == null) {
-        sys.error(s"carbon.kettle.home is not set")
-      }
-      CarbonDataRDDFactory.loadCarbonData(
-        sqlContext,
-        carbonLoadModel,
-        storeLocation,
-        relation.cubeMeta.storePath,
-        kettleHomePath,
-        relation.cubeMeta.partitioner, columinar, isAgg = true)
-    }
-  }
-
   def configSplitMaxSize(context: SparkContext, filePaths: String,
     hadoopConfiguration: Configuration): Unit = {
     val defaultParallelism = if (context.defaultParallelism < 1) 1 else context.defaultParallelism

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala
index 7dcb33b..5993677 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala
@@ -25,8 +25,9 @@ import org.carbondata.core.iterator.CarbonIterator
 import org.carbondata.query.carbon.executor.QueryExecutorFactory
 import org.carbondata.query.carbon.model.QueryModel
 import org.carbondata.query.carbon.result.BatchRawResult
+import org.carbondata.query.carbon.result.iterator.ChunkRawRowIterartor
 import org.carbondata.query.expression.Expression
-import org.carbondata.spark.RawKeyVal
+import org.carbondata.spark.{RawKey, RawKeyVal}
 
 
 /**
@@ -48,7 +49,7 @@ class CarbonRawQueryRDD[K, V](
     sc: SparkContext,
     queryModel: QueryModel,
     filterExpression: Expression,
-    keyClass: RawKeyVal[K, V],
+    keyClass: RawKey[K, V],
     @transient conf: Configuration,
     cubeCreationTime: Long,
     schemaLastUpdatedTime: Long,
@@ -66,7 +67,7 @@ class CarbonRawQueryRDD[K, V](
   override def compute(thepartition: Partition, context: TaskContext): Iterator[(K, V)] = {
     val LOGGER = LogServiceFactory.getLogService(this.getClass().getName());
     val iter = new Iterator[(K, V)] {
-      var rowIterator: CarbonIterator[BatchRawResult] = _
+      var rowIterator: CarbonIterator[Array[Any]] = _
       var queryStartTime: Long = 0
       try {
         val carbonSparkPartition = thepartition.asInstanceOf[CarbonSparkPartition]
@@ -83,8 +84,9 @@ class CarbonRawQueryRDD[K, V](
               System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties");
           }
           // execute query
-          rowIterator = QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel)
-            .asInstanceOf[CarbonIterator[BatchRawResult]]
+          rowIterator = new ChunkRawRowIterartor(
+            QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel)
+            .asInstanceOf[CarbonIterator[BatchRawResult]]).asInstanceOf[CarbonIterator[Array[Any]]]
         }
       } catch {
         case e: Exception =>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
index 6cd9986..9bc1a64 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
@@ -105,18 +105,22 @@ object CarbonScalaUtil {
     }
   }
 
+  def convertValueToSparkDataType(value: Any,
+      dataType: org.apache.spark.sql.types.DataType): Any = {
+    dataType match {
+      case StringType => value.toString
+      case IntegerType => value.toString.toInt
+      case LongType => value.toString.toLong
+      case DoubleType => value.toString.toDouble
+      case FloatType => value.toString.toFloat
+      case _ => value.toString.toDouble
+    }
+  }
+
 
   case class TransformHolder(rdd: Any, mataData: CarbonMetaData)
 
   object CarbonSparkUtil {
-    def createBaseRDD(carbonContext: CarbonContext, carbonTable: CarbonTable): TransformHolder = {
-      val relation = CarbonEnv.getInstance(carbonContext).carbonCatalog
-        .lookupRelation1(Option(carbonTable.getDatabaseName),
-          carbonTable.getFactTableName, None)(carbonContext).asInstanceOf[CarbonRelation]
-      val rdd = new SchemaRDD(carbonContext, relation)
-      rdd.registerTempTable(carbonTable.getFactTableName)
-      TransformHolder(rdd, createSparkMeta(carbonTable))
-    }
 
     def createSparkMeta(carbonTable: CarbonTable): CarbonMetaData = {
       val dimensionsAttr = carbonTable.getDimensionByTableName(carbonTable.getFactTableName)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
index 121a3a5..ebc8ead 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -395,7 +395,7 @@ object GlobalDictionaryUtil extends Logging {
 
     // update CarbonDataLoadSchema
     val carbonTable = catalog.lookupRelation1(Option(model.table.getDatabaseName),
-          model.table.getTableName, None)(sqlContext)
+          model.table.getTableName)(sqlContext)
         .asInstanceOf[CarbonRelation].cubeMeta.carbonTable
     carbonLoadModel.setCarbonDataLoadSchema(new CarbonDataLoadSchema(carbonTable))
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/resources/datawithmaxinteger.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datawithmaxinteger.csv b/integration/spark/src/test/resources/datawithmaxinteger.csv
new file mode 100644
index 0000000..52dfdfc
--- /dev/null
+++ b/integration/spark/src/test/resources/datawithmaxinteger.csv
@@ -0,0 +1,12 @@
+imei,age
+1AA1,10
+1AA2,26
+1AA3,10
+1AA4,10
+1AA5,20
+1AA6,10
+1AA7,10
+1AA8,10
+1AA9,10
+1AA10,10
+1AA12,2147483647
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/resources/datawithmaxmininteger.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datawithmaxmininteger.csv b/integration/spark/src/test/resources/datawithmaxmininteger.csv
new file mode 100644
index 0000000..5677a40
--- /dev/null
+++ b/integration/spark/src/test/resources/datawithmaxmininteger.csv
@@ -0,0 +1,13 @@
+imei,age
+1AA1,10
+1AA2,26
+1AA3,10
+1AA4,10
+1AA5,20
+1AA6,10
+1AA7,10
+1AA8,10
+1AA9,10
+1AA10,10
+1AA11,-2147483648
+1AA12,2147483647
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/resources/datawithmininteger.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datawithmininteger.csv b/integration/spark/src/test/resources/datawithmininteger.csv
new file mode 100644
index 0000000..cc34efa
--- /dev/null
+++ b/integration/spark/src/test/resources/datawithmininteger.csv
@@ -0,0 +1,12 @@
+imei,age
+1AA1,10
+1AA2,26
+1AA3,10
+1AA4,10
+1AA5,20
+1AA6,10
+1AA7,10
+1AA8,10
+1AA9,10
+1AA10,10
+1AA11,-2147483648
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
index eaa249e..5811d3a 100644
--- a/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
@@ -19,12 +19,12 @@ package org.apache.spark.sql.common.util
 
 import java.util.{Locale, TimeZone}
 
-import org.apache.spark.sql.{DataFrame, Row, SQLContext}
+import scala.collection.JavaConversions._
+
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.columnar.InMemoryRelation
-
-import scala.collection.JavaConversions._
+import org.apache.spark.sql.execution.columnar.InMemoryRelation
+import org.apache.spark.sql.{DataFrame, Row, SQLContext}
 
 class QueryTest extends PlanTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/test/scala/org/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithMaxMinInteger.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithMaxMinInteger.scala b/integration/spark/src/test/scala/org/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithMaxMinInteger.scala
new file mode 100644
index 0000000..26c88f8
--- /dev/null
+++ b/integration/spark/src/test/scala/org/carbondata/integration/spark/testsuite/dataload/TestLoadDataWithMaxMinInteger.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.carbondata.integration.spark.testsuite.dataload
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
+/**
+ * Test Class for data loading when there are min integer value in int column
+ *
+ */
+class TestLoadDataWithMaxMinInteger extends QueryTest with BeforeAndAfterAll {
+  override def beforeAll {
+    sql("drop table if exists integer_table_01")
+    sql("drop table if exists integer_table_02")
+    sql("drop table if exists integer_table_03")
+  }
+  test("test carbon table data loading when the int column " +
+    "contains min integer value") {
+    sql(
+      """
+        CREATE TABLE integer_table_01(imei string,age int)
+        STORED BY 'org.apache.carbondata.format'
+      """)
+    sql(
+      """
+        LOAD DATA INPATH './src/test/resources/datawithmininteger.csv'
+        INTO table integer_table_01 options ('DELIMITER'=',',
+        'QUOTECHAR'='"', 'FILEHEADER'= 'imei,age')
+      """)
+    checkAnswer(sql("select age from integer_table_01"),
+      Seq(Row(10.0), Row(26.0), Row(10.0), Row(10.0), Row(20.0),
+        Row(10.0), Row(10.0), Row(10.0), Row(10.0), Row(10.0),
+        Row(-2147483648.0)))
+  }
+
+  test("test carbon table data loading when the int column " +
+    "contains max integer value") {
+    sql(
+      """
+        CREATE TABLE integer_table_02(imei string,age int)
+        STORED BY 'org.apache.carbondata.format'
+      """)
+    sql(
+      """
+        LOAD DATA INPATH './src/test/resources/datawithmaxinteger.csv'
+        INTO table integer_table_02 options ('DELIMITER'=',',
+        'QUOTECHAR'='"', 'FILEHEADER'= 'imei,age')
+      """)
+    checkAnswer(sql("select age from integer_table_02"),
+      Seq(Row(10.0), Row(26.0), Row(10.0), Row(10.0), Row(20.0),
+        Row(10.0), Row(10.0), Row(10.0), Row(10.0), Row(10.0),
+        Row(2147483647.0)))
+  }
+
+  test("test carbon table data loading when the int column " +
+    "contains min and max integer value") {
+    sql(
+      """
+        CREATE TABLE integer_table_03(imei string,age int)
+        STORED BY 'org.apache.carbondata.format'
+      """)
+    sql(
+      """
+        LOAD DATA INPATH './src/test/resources/datawithmaxmininteger.csv'
+        INTO table integer_table_03 options ('DELIMITER'=',',
+        'QUOTECHAR'='"', 'FILEHEADER'= 'imei,age')
+      """)
+    checkAnswer(sql("select age from integer_table_03"),
+      Seq(Row(10.0), Row(26.0), Row(10.0), Row(10.0), Row(20.0),
+        Row(10.0), Row(10.0), Row(10.0), Row(10.0), Row(10.0),
+        Row(-2147483648.0), Row(2147483647.0)))
+  }
+  override def afterAll {
+    sql("drop table if exists integer_table_01")
+    sql("drop table if exists integer_table_02")
+    sql("drop table if exists integer_table_03")
+  }
+}


[26/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java b/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.java
new file mode 100644
index 0000000..0c61947
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/processor/impl/DataBlockIteratorImpl.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.carbondata.scan.processor.impl;
+
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.processor.AbstractDataBlockIterator;
+import org.carbondata.scan.result.Result;
+
+/**
+ * Below class will be used to process the block for detail query
+ */
+public class DataBlockIteratorImpl extends AbstractDataBlockIterator {
+
+  /**
+   * DataBlockIteratorImpl Constructor
+   *
+   * @param blockExecutionInfo execution information
+   */
+  public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo,
+      FileHolder fileReader, int batchSize) {
+    super(blockExecutionInfo, fileReader, batchSize);
+  }
+
+  /**
+   * It scans the block and returns the result with @batchSize
+   *
+   * @return Result of @batchSize
+   */
+  public Result next() {
+    this.scannerResultAggregator.collectData(scannedResult, batchSize);
+    Result result = this.scannerResultAggregator.getCollectedResult();
+    while (result.size() < batchSize && hasNext()) {
+      this.scannerResultAggregator.collectData(scannedResult, batchSize-result.size());
+      result.merge(this.scannerResultAggregator.getCollectedResult());
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java b/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java
new file mode 100644
index 0000000..5d00e32
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/AbstractScannedResult.java
@@ -0,0 +1,347 @@
+/*
+ * 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.carbondata.scan.result;
+
+import java.math.BigDecimal;
+import java.util.Map;
+
+import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
+
+/**
+ * Scanned result class which will store and provide the result on request
+ */
+public abstract class AbstractScannedResult {
+
+  /**
+   * current row number
+   */
+  protected int currentRow = -1;
+  /**
+   * row mapping indexes
+   */
+  protected int[] rowMapping;
+  /**
+   * key size of the fixed length column
+   */
+  private int fixedLengthKeySize;
+  /**
+   * total number of rows
+   */
+  private int totalNumberOfRows;
+  /**
+   * to keep track of number of rows process
+   */
+  private int rowCounter;
+  /**
+   * dimension column data chunk
+   */
+  private DimensionColumnDataChunk[] dataChunks;
+  /**
+   * measure column data chunk
+   */
+  private MeasureColumnDataChunk[] measureDataChunks;
+  /**
+   * dictionary column block index in file
+   */
+  private int[] dictionaryColumnBlockIndexes;
+
+  /**
+   * no dictionary column block index in file
+   */
+  private int[] noDictionaryColumnBlockIndexes;
+
+  /**
+   * column group to is key structure info
+   * which will be used to get the key from the complete
+   * column group key
+   * For example if only one dimension of the column group is selected
+   * then from complete column group key it will be used to mask the key and
+   * get the particular column key
+   */
+  private Map<Integer, KeyStructureInfo> columnGroupKeyStructureInfo;
+
+  public AbstractScannedResult(BlockExecutionInfo blockExecutionInfo) {
+    this.fixedLengthKeySize = blockExecutionInfo.getFixedLengthKeySize();
+    this.noDictionaryColumnBlockIndexes = blockExecutionInfo.getNoDictionaryBlockIndexes();
+    this.dictionaryColumnBlockIndexes = blockExecutionInfo.getDictionaryColumnBlockIndex();
+    this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo();
+  }
+
+  /**
+   * Below method will be used to set the dimension chunks
+   * which will be used to create a row
+   *
+   * @param dataChunks dimension chunks used in query
+   */
+  public void setDimensionChunks(DimensionColumnDataChunk[] dataChunks) {
+    this.dataChunks = dataChunks;
+  }
+
+  /**
+   * Below method will be used to set the measure column chunks
+   *
+   * @param measureDataChunks measure data chunks
+   */
+  public void setMeasureChunks(MeasureColumnDataChunk[] measureDataChunks) {
+    this.measureDataChunks = measureDataChunks;
+  }
+
+  /**
+   * Below method will be used to get the chunk based in measure ordinal
+   *
+   * @param ordinal measure ordinal
+   * @return measure column chunk
+   */
+  public MeasureColumnDataChunk getMeasureChunk(int ordinal) {
+    return measureDataChunks[ordinal];
+  }
+
+  /**
+   * Below method will be used to get the key for all the dictionary dimensions
+   * which is present in the query
+   *
+   * @param rowId row id selected after scanning
+   * @return return the dictionary key
+   */
+  protected byte[] getDictionaryKeyArray(int rowId) {
+    byte[] completeKey = new byte[fixedLengthKeySize];
+    int offset = 0;
+    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
+      offset += dataChunks[dictionaryColumnBlockIndexes[i]]
+          .fillChunkData(completeKey, offset, rowId,
+              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
+    }
+    rowCounter++;
+    return completeKey;
+  }
+
+  /**
+   * Just increment the counter incase of query only on measures.
+   */
+  public void incrementCounter() {
+    rowCounter ++;
+    currentRow ++;
+  }
+
+  /**
+   * Below method will be used to get the dimension data based on dimension
+   * ordinal and index
+   *
+   * @param dimOrdinal dimension ordinal present in the query
+   * @param rowId      row index
+   * @return dimension data based on row id
+   */
+  protected byte[] getDimensionData(int dimOrdinal, int rowId) {
+    return dataChunks[dimOrdinal].getChunkData(rowId);
+  }
+
+  /**
+   * Below method will be used to get the dimension key array
+   * for all the no dictionary dimension present in the query
+   *
+   * @param rowId row number
+   * @return no dictionary keys for all no dictionary dimension
+   */
+  protected byte[][] getNoDictionaryKeyArray(int rowId) {
+    byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnBlockIndexes.length][];
+    int position = 0;
+    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
+      noDictionaryColumnsKeys[position++] =
+          dataChunks[noDictionaryColumnBlockIndexes[i]].getChunkData(rowId);
+    }
+    return noDictionaryColumnsKeys;
+  }
+
+  /**
+   * Below method will be used to get the complex type keys array based
+   * on row id for all the complex type dimension selected in query
+   *
+   * @param rowId row number
+   * @return complex type key array for all the complex dimension selected in query
+   */
+  protected byte[][] getComplexTypeKeyArray(int rowId) {
+    return new byte[0][];
+  }
+
+  /**
+   * @return return the total number of row after scanning
+   */
+  public int numberOfOutputRows() {
+    return this.totalNumberOfRows;
+  }
+
+  /**
+   * to check whether any more row is present in the result
+   *
+   * @return
+   */
+  public boolean hasNext() {
+    return rowCounter < this.totalNumberOfRows;
+  }
+
+  /**
+   * As this class will be a flyweight object so
+   * for one block all the blocklet scanning will use same result object
+   * in that case we need to reset the counter to zero so
+   * for new result it will give the result from zero
+   */
+  public void reset() {
+    rowCounter = 0;
+    currentRow = -1;
+  }
+
+  /**
+   * @param totalNumberOfRows set total of number rows valid after scanning
+   */
+  public void setNumberOfRows(int totalNumberOfRows) {
+    this.totalNumberOfRows = totalNumberOfRows;
+  }
+
+  /**
+   * After applying filter it will return the  bit set with the valid row indexes
+   * so below method will be used to set the row indexes
+   *
+   * @param indexes
+   */
+  public void setIndexes(int[] indexes) {
+    this.rowMapping = indexes;
+  }
+
+  /**
+   * Below method will be used to check whether measure value is null or not
+   *
+   * @param ordinal  measure ordinal
+   * @param rowIndex row number to be checked
+   * @return whether it is null or not
+   */
+  protected boolean isNullMeasureValue(int ordinal, int rowIndex) {
+    return measureDataChunks[ordinal].getNullValueIndexHolder().getBitSet().get(rowIndex);
+  }
+
+  /**
+   * Below method will be used to get the measure value of
+   * long type
+   *
+   * @param ordinal  measure ordinal
+   * @param rowIndex row number of the measure value
+   * @return measure value of long type
+   */
+  protected long getLongMeasureValue(int ordinal, int rowIndex) {
+    return measureDataChunks[ordinal].getMeasureDataHolder().getReadableLongValueByIndex(rowIndex);
+  }
+
+  /**
+   * Below method will be used to get the measure value of double type
+   *
+   * @param ordinal  measure ordinal
+   * @param rowIndex row number
+   * @return measure value of double type
+   */
+  protected double getDoubleMeasureValue(int ordinal, int rowIndex) {
+    return measureDataChunks[ordinal].getMeasureDataHolder()
+        .getReadableDoubleValueByIndex(rowIndex);
+  }
+
+  /**
+   * Below method will be used to get the measure type of big decimal data type
+   *
+   * @param ordinal  ordinal of the of the measure
+   * @param rowIndex row number
+   * @return measure of big decimal type
+   */
+  protected BigDecimal getBigDecimalMeasureValue(int ordinal, int rowIndex) {
+    return measureDataChunks[ordinal].getMeasureDataHolder()
+        .getReadableBigDecimalValueByIndex(rowIndex);
+  }
+
+  /**
+   * will return the current valid row id
+   *
+   * @return valid row id
+   */
+  public abstract int getCurrenrRowId();
+
+  /**
+   * @return dictionary key array for all the dictionary dimension
+   * selected in query
+   */
+  public abstract byte[] getDictionaryKeyArray();
+
+  /**
+   * Return the dimension data based on dimension ordinal
+   *
+   * @param dimensionOrdinal dimension ordinal
+   * @return dimension data
+   */
+  public abstract byte[] getDimensionKey(int dimensionOrdinal);
+
+  /**
+   * Below method will be used to get the complex type key array
+   *
+   * @return complex type key array
+   */
+  public abstract byte[][] getComplexTypeKeyArray();
+
+  /**
+   * Below method will be used to get the no dictionary key
+   * array for all the no dictionary dimension selected in query
+   *
+   * @return no dictionary key array for all the no dictionary dimension
+   */
+  public abstract byte[][] getNoDictionaryKeyArray();
+
+  /**
+   * Below method will be used to to check whether measure value
+   * is null or for a measure
+   *
+   * @param ordinal measure ordinal
+   * @return is null or not
+   */
+  public abstract boolean isNullMeasureValue(int ordinal);
+
+  /**
+   * Below method will be used to get the measure value for measure
+   * of long data type
+   *
+   * @param ordinal measure ordinal
+   * @return long value of measure
+   */
+  public abstract long getLongMeasureValue(int ordinal);
+
+  /**
+   * Below method will be used to get the value of measure of double
+   * type
+   *
+   * @param ordinal measure ordinal
+   * @return measure value
+   */
+  public abstract double getDoubleMeasureValue(int ordinal);
+
+  /**
+   * Below method will be used to get the data of big decimal type
+   * of a measure
+   *
+   * @param ordinal measure ordinal
+   * @return measure value
+   */
+  public abstract BigDecimal getBigDecimalMeasureValue(int ordinal);
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java b/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java
new file mode 100644
index 0000000..c13b0f7
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/BatchRawResult.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.result;
+
+/**
+ * Below class holds the query result of batches.
+ */
+public class BatchRawResult extends BatchResult {
+
+  /**
+   * This method will return one row at a time based on the counter given.
+   * @param counter
+   * @return
+   */
+  public Object[] getRawRow(int counter) {
+    return rows[counter];
+  }
+
+  /**
+   * For getting the total size.
+   * @return
+   */
+  public int getSize() {
+    return rows.length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/BatchResult.java b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
new file mode 100644
index 0000000..dc14060
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/BatchResult.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.carbondata.scan.result;
+
+import java.util.NoSuchElementException;
+
+import org.carbondata.core.iterator.CarbonIterator;
+
+/**
+ * Below class holds the query result
+ */
+public class BatchResult extends CarbonIterator<Object[]> {
+
+  /**
+   * list of keys
+   */
+  protected Object[][] rows;
+
+  /**
+   * counter to check whether all the records are processed or not
+   */
+  protected int counter;
+
+  public BatchResult() {
+    this.rows = new Object[0][];
+  }
+
+  /**
+   * Below method will be used to get the rows
+   *
+   * @return
+   */
+  public Object[][] getRows() {
+    return rows;
+  }
+
+  /**
+   * Below method will be used to get the set the values
+   *
+   * @param rows
+   */
+  public void setRows(Object[][] rows) {
+    this.rows = rows;
+  }
+
+
+  /**
+   * Returns {@code true} if the iteration has more elements.
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  @Override public boolean hasNext() {
+    return counter < rows.length;
+  }
+
+  /**
+   * Returns the next element in the iteration.
+   *
+   * @return the next element in the iteration
+   */
+  @Override public Object[] next() {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
+    Object[] row = rows[counter];
+    counter++;
+    return row;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.java b/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.java
new file mode 100644
index 0000000..f3085ce
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/ListBasedResultWrapper.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.carbondata.scan.result;
+
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+public class ListBasedResultWrapper {
+
+  private ByteArrayWrapper key;
+
+  private Object[] value;
+
+  /**
+   * @return the key
+   */
+  public ByteArrayWrapper getKey() {
+    return key;
+  }
+
+  /**
+   * @param key the key to set
+   */
+  public void setKey(ByteArrayWrapper key) {
+    this.key = key;
+  }
+
+  /**
+   * @return the value
+   */
+  public Object[] getValue() {
+    return value;
+  }
+
+  /**
+   * @param value the value to set
+   */
+  public void setValue(Object[] value) {
+    this.value = value;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/Result.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/Result.java b/core/src/main/java/org/carbondata/scan/result/Result.java
new file mode 100644
index 0000000..98466bb
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/Result.java
@@ -0,0 +1,70 @@
+/*
+ * 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.carbondata.scan.result;
+
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * Result interface for storing the result
+ */
+public interface Result<K, V> {
+  /**
+   * Below method will be used to
+   * add the sccaed result
+   *
+   * @param result
+   */
+  void addScannedResult(K result);
+
+  /**
+   * Returns {@code true} if the iteration has more elements.
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  boolean hasNext();
+
+  /**
+   * Below method will return the result key
+   *
+   * @return key
+   */
+  ByteArrayWrapper getKey();
+
+  /**
+   * Below code will return the result value
+   *
+   * @return value
+   */
+  V[] getValue();
+
+  void merge(Result<K, V> otherResult);
+
+  /**
+   * Below method will be used to get the result
+   *
+   * @return
+   */
+  K getResult();
+
+  /**
+   * @return size of the result
+   */
+  int size();
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java b/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java
new file mode 100644
index 0000000..962d9a3
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/impl/FilterQueryScannedResult.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.result.impl;
+
+import java.math.BigDecimal;
+
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.result.AbstractScannedResult;
+
+/**
+ * Result provider class in case of filter query
+ * In case of filter query data will be send
+ * based on filtered row index
+ */
+public class FilterQueryScannedResult extends AbstractScannedResult {
+
+  public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) {
+    super(tableBlockExecutionInfos);
+  }
+
+  /**
+   * @return dictionary key array for all the dictionary dimension
+   * selected in query
+   */
+  @Override public byte[] getDictionaryKeyArray() {
+    ++currentRow;
+    return getDictionaryKeyArray(rowMapping[currentRow]);
+  }
+
+  /**
+   * Below method will be used to get the complex type key array
+   *
+   * @return complex type key array
+   */
+  @Override public byte[][] getComplexTypeKeyArray() {
+    return getComplexTypeKeyArray(rowMapping[currentRow]);
+  }
+
+  /**
+   * Below method will be used to get the no dictionary key
+   * array for all the no dictionary dimension selected in query
+   *
+   * @return no dictionary key array for all the no dictionary dimension
+   */
+  @Override public byte[][] getNoDictionaryKeyArray() {
+    return getNoDictionaryKeyArray(rowMapping[currentRow]);
+  }
+
+  /**
+   * will return the current valid row id
+   *
+   * @return valid row id
+   */
+  @Override public int getCurrenrRowId() {
+    return rowMapping[currentRow];
+  }
+
+  /**
+   * Return the dimension data based on dimension ordinal
+   *
+   * @param dimensionOrdinal dimension ordinal
+   * @return dimension data
+   */
+  @Override public byte[] getDimensionKey(int dimensionOrdinal) {
+    return getDimensionData(dimensionOrdinal, rowMapping[currentRow]);
+  }
+
+  /**
+   * Below method will be used to to check whether measure value
+   * is null or for a measure
+   *
+   * @param ordinal measure ordinal
+   * @return is null or not
+   */
+  @Override public boolean isNullMeasureValue(int ordinal) {
+    return isNullMeasureValue(ordinal, rowMapping[currentRow]);
+  }
+
+  /**
+   * Below method will be used to get the measure value for measure
+   * of long data type
+   *
+   * @param ordinal measure ordinal
+   * @return long value of measure
+   */
+  @Override public long getLongMeasureValue(int ordinal) {
+    return getLongMeasureValue(ordinal, rowMapping[currentRow]);
+  }
+
+  /**
+   * Below method will be used to get the value of measure of double
+   * type
+   *
+   * @param ordinal measure ordinal
+   * @return measure value
+   */
+  @Override public double getDoubleMeasureValue(int ordinal) {
+    return getDoubleMeasureValue(ordinal, rowMapping[currentRow]);
+  }
+
+  /**
+   * Below method will be used to get the data of big decimal type
+   * of a measure
+   *
+   * @param ordinal measure ordinal
+   * @return measure value
+   */
+  @Override public BigDecimal getBigDecimalMeasureValue(int ordinal) {
+    return getBigDecimalMeasureValue(ordinal, rowMapping[currentRow]);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java b/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java
new file mode 100644
index 0000000..24ebf5b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/impl/ListBasedResult.java
@@ -0,0 +1,153 @@
+/*
+ * 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.carbondata.scan.result.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.scan.result.ListBasedResultWrapper;
+import org.carbondata.scan.result.Result;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * Below class is a holder over list based result wrapper
+ */
+public class ListBasedResult implements Result<List<ListBasedResultWrapper>, Object> {
+
+  /**
+   * current result list
+   */
+  private List<ListBasedResultWrapper> currentRowPointer;
+
+  /**
+   * all result list , this is required because if we merger all the scanned
+   * result from all the blocks in one list, that list creation will take more
+   * time as every time list will create a big array and then it will do copy
+   * the older element to new array, and creation of big array will also be a
+   * problem if memory is fragmented then jvm in to do defragmentation to
+   * create a big space, but if divide the data in multiple list than it avoid
+   * copy and defragmentation
+   */
+  private List<List<ListBasedResultWrapper>> allRowsResult;
+
+  /**
+   * counter to check how many result processed
+   */
+  private int totalRecordCounter = -1;
+
+  /**
+   * number of records
+   */
+  private int totalNumberOfRecords;
+
+  /**
+   * current counter of the record in list
+   */
+  private int listRecordCounter = -1;
+
+  /**
+   * current list counter
+   */
+  private int currentListCounter;
+
+  public ListBasedResult() {
+    currentRowPointer =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    allRowsResult =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * below method will be used to add the scan result
+   */
+  @Override public void addScannedResult(List<ListBasedResultWrapper> listBasedResult) {
+    this.currentRowPointer = listBasedResult;
+    totalNumberOfRecords = listBasedResult.size();
+    allRowsResult.add(listBasedResult);
+  }
+
+  /**
+   * Method to check more result is present
+   * or not
+   */
+  @Override public boolean hasNext() {
+    if (allRowsResult.size() == 0) {
+      return false;
+    }
+    // As we are storing data in list of list, below code is to check whether
+    // any more result is present
+    // in the result.
+    // first it will check list counter is zero if it is zero
+    // than it will check list counter to check how many list has been processed
+    // if more list are present and all the list of current list is processed
+    // than it will take a new list from all row result list
+    totalRecordCounter++;
+    listRecordCounter++;
+    if (listRecordCounter == 0 || (listRecordCounter >= currentRowPointer.size()
+        && currentListCounter < allRowsResult.size())) {
+      listRecordCounter = 0;
+      currentRowPointer = allRowsResult.get(currentListCounter);
+      currentListCounter++;
+    }
+    return totalRecordCounter < totalNumberOfRecords;
+  }
+
+  /**
+   * @return key
+   */
+  @Override public ByteArrayWrapper getKey() {
+    return currentRowPointer.get(listRecordCounter).getKey();
+  }
+
+  /**
+   * @return will return the value
+   */
+  @Override public Object[] getValue() {
+    return currentRowPointer.get(listRecordCounter).getValue();
+  }
+
+  /***
+   * below method will be used to merge the
+   * scanned result
+   *
+   * @param otherResult return to be merged
+   */
+  @Override public void merge(Result<List<ListBasedResultWrapper>, Object> otherResult) {
+    if (otherResult.size() > 0) {
+      totalNumberOfRecords += otherResult.size();
+      this.allRowsResult.add(otherResult.getResult());
+    }
+  }
+
+  /**
+   * Return the size of the result
+   */
+  @Override public int size() {
+    return totalNumberOfRecords;
+  }
+
+  /**
+   * @return the complete result
+   */
+  @Override public List<ListBasedResultWrapper> getResult() {
+    return currentRowPointer;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java b/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java
new file mode 100644
index 0000000..9782099
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/impl/NonFilterQueryScannedResult.java
@@ -0,0 +1,109 @@
+package org.carbondata.scan.result.impl;
+
+import java.math.BigDecimal;
+
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.result.AbstractScannedResult;
+
+/**
+ * Result provide class for non filter query
+ * In case of no filter query we need to return
+ * complete data
+ */
+public class NonFilterQueryScannedResult extends AbstractScannedResult {
+
+  public NonFilterQueryScannedResult(BlockExecutionInfo blockExecutionInfo) {
+    super(blockExecutionInfo);
+  }
+
+  /**
+   * @return dictionary key array for all the dictionary dimension selected in
+   * query
+   */
+  @Override public byte[] getDictionaryKeyArray() {
+    ++currentRow;
+    return getDictionaryKeyArray(currentRow);
+  }
+
+  /**
+   * Below method will be used to get the complex type key array
+   *
+   * @return complex type key array
+   */
+  @Override public byte[][] getComplexTypeKeyArray() {
+    return getComplexTypeKeyArray(currentRow);
+  }
+
+  /**
+   * Below method will be used to get the no dictionary key array for all the
+   * no dictionary dimension selected in query
+   *
+   * @return no dictionary key array for all the no dictionary dimension
+   */
+  @Override public byte[][] getNoDictionaryKeyArray() {
+    return getNoDictionaryKeyArray(currentRow);
+  }
+
+  /**
+   * will return the current valid row id
+   *
+   * @return valid row id
+   */
+  @Override public int getCurrenrRowId() {
+    return currentRow;
+  }
+
+  /**
+   * Return the dimension data based on dimension ordinal
+   *
+   * @param dimensionOrdinal dimension ordinal
+   * @return dimension data
+   */
+  @Override public byte[] getDimensionKey(int dimensionOrdinal) {
+    return getDimensionData(dimensionOrdinal, currentRow);
+  }
+
+  /**
+   * Below method will be used to to check whether measure value is null or
+   * for a measure
+   *
+   * @param ordinal measure ordinal
+   * @return is null or not
+   */
+  @Override public boolean isNullMeasureValue(int ordinal) {
+    return isNullMeasureValue(ordinal, currentRow);
+  }
+
+  /**
+   * Below method will be used to get the measure value for measure of long
+   * data type
+   *
+   * @param ordinal measure ordinal
+   * @return long value of measure
+   */
+  @Override public long getLongMeasureValue(int ordinal) {
+    return getLongMeasureValue(ordinal, currentRow);
+  }
+
+  /**
+   * Below method will be used to get the value of measure of double type
+   *
+   * @param ordinal measure ordinal
+   * @return measure value
+   */
+  @Override public double getDoubleMeasureValue(int ordinal) {
+    return getDoubleMeasureValue(ordinal, currentRow);
+  }
+
+  /**
+   * Below method will be used to get the data of big decimal type of a
+   * measure
+   *
+   * @param ordinal measure ordinal
+   * @return measure value
+   */
+  @Override public BigDecimal getBigDecimalMeasureValue(int ordinal) {
+    return getBigDecimalMeasureValue(ordinal, currentRow);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
new file mode 100644
index 0000000..2356a9f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.result.iterator;
+
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.carbon.datastore.DataRefNodeFinder;
+import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.core.util.CarbonProperties;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.processor.AbstractDataBlockIterator;
+import org.carbondata.scan.processor.impl.DataBlockIteratorImpl;
+
+/**
+ * In case of detail query we cannot keep all the records in memory so for
+ * executing that query are returning a iterator over block and every time next
+ * call will come it will execute the block and return the result
+ */
+public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
+
+  /**
+   * LOGGER.
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractDetailQueryResultIterator.class.getName());
+
+  /**
+   * execution info of the block
+   */
+  protected List<BlockExecutionInfo> blockExecutionInfos;
+
+  /**
+   * number of cores which can be used
+   */
+  private int batchSize;
+
+  /**
+   * file reader which will be used to execute the query
+   */
+  protected FileHolder fileReader;
+
+  protected AbstractDataBlockIterator dataBlockIterator;
+
+  protected boolean nextBatch = false;
+
+  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
+    String batchSizeString =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE);
+    if (null != batchSizeString) {
+      try {
+        batchSize = Integer.parseInt(batchSizeString);
+      } catch (NumberFormatException ne) {
+        LOGGER.error("Invalid inmemory records size. Using default value");
+        batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
+      }
+    } else {
+      batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
+    }
+
+    this.blockExecutionInfos = infos;
+    this.fileReader = FileFactory.getFileHolder(
+        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
+    intialiseInfos();
+  }
+
+  private void intialiseInfos() {
+    for (BlockExecutionInfo blockInfo : blockExecutionInfos) {
+      DataRefNodeFinder finder = new BTreeDataRefNodeFinder(blockInfo.getEachColumnValueSize());
+      DataRefNode startDataBlock = finder
+          .findFirstDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getStartKey());
+      DataRefNode endDataBlock = finder
+          .findLastDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getEndKey());
+      long numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
+      blockInfo.setFirstDataBlock(startDataBlock);
+      blockInfo.setNumberOfBlockToScan(numberOfBlockToScan);
+    }
+  }
+
+  @Override public boolean hasNext() {
+    if ((dataBlockIterator != null && dataBlockIterator.hasNext()) || nextBatch) {
+      return true;
+    } else {
+      dataBlockIterator = getDataBlockIterator();
+      while (dataBlockIterator != null) {
+        if (dataBlockIterator.hasNext()) {
+          return true;
+        }
+        dataBlockIterator = getDataBlockIterator();
+      }
+      return false;
+    }
+  }
+
+  private DataBlockIteratorImpl getDataBlockIterator() {
+    if(blockExecutionInfos.size() > 0) {
+      BlockExecutionInfo executionInfo = blockExecutionInfos.get(0);
+      blockExecutionInfos.remove(executionInfo);
+      return new DataBlockIteratorImpl(executionInfo, fileReader, batchSize);
+    }
+    return null;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
new file mode 100644
index 0000000..5cc4f1e
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.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.carbondata.scan.result.iterator;
+
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.scan.result.BatchResult;
+
+/**
+ * Iterator over row result
+ */
+public class ChunkRowIterator extends CarbonIterator<Object[]> {
+
+  /**
+   * iterator over chunk result
+   */
+  private CarbonIterator<BatchResult> iterator;
+
+  /**
+   * currect chunk
+   */
+  private BatchResult currentchunk;
+
+  public ChunkRowIterator(CarbonIterator<BatchResult> iterator) {
+    this.iterator = iterator;
+    if (iterator.hasNext()) {
+      currentchunk = iterator.next();
+    }
+  }
+
+  /**
+   * Returns {@code true} if the iteration has more elements. (In other words,
+   * returns {@code true} if {@link #next} would return an element rather than
+   * throwing an exception.)
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  @Override public boolean hasNext() {
+    if (null != currentchunk) {
+      if ((currentchunk.hasNext())) {
+        return true;
+      } else if (!currentchunk.hasNext()) {
+        while (iterator.hasNext()) {
+          currentchunk = iterator.next();
+          if (currentchunk != null && currentchunk.hasNext()) {
+            return true;
+          }
+        }
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Returns the next element in the iteration.
+   *
+   * @return the next element in the iteration
+   */
+  @Override public Object[] next() {
+    return currentchunk.next();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java
new file mode 100644
index 0000000..4eb50ac
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/DetailQueryResultIterator.java
@@ -0,0 +1,84 @@
+/*
+ * 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.carbondata.scan.result.iterator;
+
+import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.result.BatchResult;
+import org.carbondata.scan.result.ListBasedResultWrapper;
+import org.carbondata.scan.result.preparator.QueryResultPreparator;
+
+/**
+ * In case of detail query we cannot keep all the records in memory so for
+ * executing that query are returning a iterator over block and every time next
+ * call will come it will execute the block and return the result
+ */
+public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator {
+
+  /**
+   * to prepare the result
+   */
+  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
+
+  private ExecutorService execService = Executors.newFixedThreadPool(1);
+
+  private Future<BatchResult> future;
+
+  public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
+      QueryResultPreparator queryResultPreparator) {
+    super(infos, queryModel);
+    this.queryResultPreparator = queryResultPreparator;
+  }
+
+  @Override public BatchResult next() {
+    BatchResult result;
+    try {
+      if (future == null) {
+        future = execute();
+      }
+      result = future.get();
+      nextBatch = false;
+      if (hasNext()) {
+        nextBatch = true;
+        future = execute();
+      } else {
+        fileReader.finish();
+      }
+    } catch (Exception ex) {
+      fileReader.finish();
+      throw new RuntimeException(ex.getCause().getMessage());
+    }
+    return result;
+  }
+
+  private Future<BatchResult> execute() {
+    return execService.submit(new Callable<BatchResult>() {
+      @Override public BatchResult call() throws QueryExecutionException {
+        return queryResultPreparator.prepareQueryResult(dataBlockIterator.next());
+      }
+    });
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
new file mode 100644
index 0000000..8c028b2
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
@@ -0,0 +1,167 @@
+/*
+ * 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.carbondata.scan.result.iterator;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.scan.result.BatchRawResult;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * This is a wrapper iterator over the detail raw query iterator.
+ * This iterator will handle the processing of the raw rows.
+ * This will handle the batch results and will iterate on the batches and give single row.
+ */
+public class RawResultIterator extends CarbonIterator<Object[]> {
+
+  private final SegmentProperties sourceSegProperties;
+
+  private final SegmentProperties destinationSegProperties;
+  /**
+   * Iterator of the Batch raw result.
+   */
+  private CarbonIterator<BatchRawResult> detailRawQueryResultIterator;
+
+  /**
+   * Counter to maintain the row counter.
+   */
+  private int counter = 0;
+
+  private Object[] currentConveretedRawRow = null;
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(RawResultIterator.class.getName());
+
+  /**
+   * batch of the result.
+   */
+  private BatchRawResult batch;
+
+  public RawResultIterator(CarbonIterator<BatchRawResult> detailRawQueryResultIterator,
+      SegmentProperties sourceSegProperties, SegmentProperties destinationSegProperties) {
+    this.detailRawQueryResultIterator = detailRawQueryResultIterator;
+    this.sourceSegProperties = sourceSegProperties;
+    this.destinationSegProperties = destinationSegProperties;
+  }
+
+  @Override public boolean hasNext() {
+
+    if (null == batch || checkIfBatchIsProcessedCompletely(batch)) {
+      if (detailRawQueryResultIterator.hasNext()) {
+        batch = detailRawQueryResultIterator.next();
+        counter = 0; // batch changed so reset the counter.
+      } else {
+        return false;
+      }
+    }
+
+    if (!checkIfBatchIsProcessedCompletely(batch)) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  @Override public Object[] next() {
+    if (null == batch) { // for 1st time
+      batch = detailRawQueryResultIterator.next();
+    }
+    if (!checkIfBatchIsProcessedCompletely(batch)) {
+      try {
+        if(null != currentConveretedRawRow){
+          counter++;
+          Object[] currentConveretedRawRowTemp = this.currentConveretedRawRow;
+          currentConveretedRawRow = null;
+          return currentConveretedRawRowTemp;
+        }
+        return convertRow(batch.getRawRow(counter++));
+      } catch (KeyGenException e) {
+        LOGGER.error(e.getMessage());
+        return null;
+      }
+    } else { // completed one batch.
+      batch = detailRawQueryResultIterator.next();
+      counter = 0;
+    }
+    try {
+      if(null != currentConveretedRawRow){
+        counter++;
+        Object[] currentConveretedRawRowTemp = this.currentConveretedRawRow;
+        currentConveretedRawRow = null;
+        return currentConveretedRawRowTemp;
+      }
+
+      return convertRow(batch.getRawRow(counter++));
+    } catch (KeyGenException e) {
+      LOGGER.error(e.getMessage());
+      return null;
+    }
+
+  }
+
+  /**
+   * for fetching the row with out incrementing counter.
+   * @return
+   */
+  public Object[] fetchConverted() throws KeyGenException {
+    if(null != currentConveretedRawRow){
+      return currentConveretedRawRow;
+    }
+    if(hasNext())
+    {
+      Object[] rawRow = batch.getRawRow(counter);
+      currentConveretedRawRow = convertRow(rawRow);;
+      return currentConveretedRawRow;
+    }
+    else
+    {
+      return null;
+    }
+  }
+
+  private Object[] convertRow(Object[] rawRow) throws KeyGenException {
+    byte[] dims = ((ByteArrayWrapper) rawRow[0]).getDictionaryKey();
+    long[] keyArray = sourceSegProperties.getDimensionKeyGenerator().getKeyArray(dims);
+    byte[] covertedBytes =
+        destinationSegProperties.getDimensionKeyGenerator().generateKey(keyArray);
+    ((ByteArrayWrapper) rawRow[0]).setDictionaryKey(covertedBytes);
+    return rawRow;
+  }
+
+  /**
+   * To check if the batch is processed completely
+   * @param batch
+   * @return
+   */
+  private boolean checkIfBatchIsProcessedCompletely(BatchRawResult batch){
+    if(counter < batch.getSize())
+    {
+      return false;
+    }
+    else{
+      return true;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java b/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java
new file mode 100644
index 0000000..7ef5b6d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/preparator/QueryResultPreparator.java
@@ -0,0 +1,10 @@
+package org.carbondata.scan.result.preparator;
+
+import org.carbondata.scan.result.BatchResult;
+import org.carbondata.scan.result.Result;
+
+public interface QueryResultPreparator<K, V> {
+
+  public BatchResult prepareQueryResult(Result<K, V> scannedResult);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java b/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java
new file mode 100644
index 0000000..a42dc67
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/preparator/impl/AbstractQueryResultPreparator.java
@@ -0,0 +1,87 @@
+package org.carbondata.scan.result.preparator.impl;
+
+import java.util.List;
+
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.executor.impl.QueryExecutorProperties;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.result.BatchResult;
+import org.carbondata.scan.result.preparator.QueryResultPreparator;
+import org.carbondata.scan.util.DataTypeUtil;
+
+public abstract class AbstractQueryResultPreparator<K, V> implements QueryResultPreparator<K, V> {
+
+  /**
+   * query properties
+   */
+  protected QueryExecutorProperties queryExecuterProperties;
+
+  /**
+   * query model
+   */
+  protected QueryModel queryModel;
+
+  public AbstractQueryResultPreparator(QueryExecutorProperties executerProperties,
+      QueryModel queryModel) {
+    this.queryExecuterProperties = executerProperties;
+    this.queryModel = queryModel;
+  }
+
+  protected void fillDimensionData(Object[][] convertedResult, List<QueryDimension> queryDimensions,
+      int dimensionCount, Object[] row, int rowIndex) {
+    QueryDimension queryDimension;
+    for (int i = 0; i < dimensionCount; i++) {
+      queryDimension = queryDimensions.get(i);
+      if (!CarbonUtil
+          .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)) {
+        row[queryDimension.getQueryOrder()] = convertedResult[i][rowIndex];
+      } else if (CarbonUtil
+          .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DIRECT_DICTIONARY)) {
+        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(queryDimension.getDimension().getDataType());
+        row[queryDimension.getQueryOrder()] = directDictionaryGenerator
+            .getValueFromSurrogate((Integer) convertedResult[i][rowIndex]);
+      } else {
+        if (queryExecuterProperties.sortDimIndexes[i] == 1) {
+          row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
+              queryExecuterProperties.columnToDictionayMapping
+                  .get(queryDimension.getDimension().getColumnId())
+                  .getDictionaryValueFromSortedIndex((Integer) convertedResult[i][rowIndex]),
+              queryDimension.getDimension().getDataType());
+        } else {
+          row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
+              queryExecuterProperties.columnToDictionayMapping
+                  .get(queryDimension.getDimension().getColumnId())
+                  .getDictionaryValueForKey((Integer) convertedResult[i][rowIndex]),
+              queryDimension.getDimension().getDataType());
+        }
+      }
+    }
+  }
+
+  protected Object[][] encodeToRows(Object[][] data) {
+    if (data.length == 0) {
+      return data;
+    }
+    Object[][] rData = new Object[data[0].length][data.length];
+    int len = data.length;
+    for (int i = 0; i < rData.length; i++) {
+      for (int j = 0; j < len; j++) {
+        rData[i][j] = data[j][i];
+      }
+    }
+    return rData;
+  }
+
+  protected BatchResult getEmptyChunkResult(int size) {
+    Object[][] row = new Object[size][1];
+    BatchResult chunkResult = new BatchResult();
+    chunkResult.setRows(row);
+    return chunkResult;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java
new file mode 100644
index 0000000..17735bc
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/preparator/impl/DetailQueryResultPreparatorImpl.java
@@ -0,0 +1,139 @@
+/*
+ * 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.carbondata.scan.result.preparator.impl;
+
+import java.nio.charset.Charset;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.executor.impl.QueryExecutorProperties;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.result.BatchResult;
+import org.carbondata.scan.result.ListBasedResultWrapper;
+import org.carbondata.scan.result.Result;
+import org.carbondata.scan.util.DataTypeUtil;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * Below class will be used to get the result by converting to actual data
+ * Actual data conversion can be converting the surrogate key to actual data
+ *
+ * @TODO there are many things in class which is very confusing, need to check
+ * why it was handled like that and how we can handle that in a better
+ * way.Need to revisit this class. IF aggregation is push down to spark
+ * layer and if we can process the data in byte array format then this
+ * class wont be useful so in future we can delete this class.
+ * @TODO need to expose one interface which will return the result based on required type
+ * for example its implementation case return converted result or directly result with out
+ * converting to actual value
+ */
+public class DetailQueryResultPreparatorImpl
+    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, Object> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DetailQueryResultPreparatorImpl.class.getName());
+
+  public DetailQueryResultPreparatorImpl(QueryExecutorProperties executerProperties,
+      QueryModel queryModel) {
+    super(executerProperties, queryModel);
+  }
+
+  @Override public BatchResult prepareQueryResult(
+      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
+    if ((null == scannedResult || scannedResult.size() < 1)) {
+      return new BatchResult();
+    }
+    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
+    int dimensionCount = queryDimension.size();
+    int totalNumberOfColumn = dimensionCount + queryExecuterProperties.measureDataTypes.length;
+    Object[][] resultData = new Object[scannedResult.size()][totalNumberOfColumn];
+    if (!queryExecuterProperties.isFunctionQuery && totalNumberOfColumn == 0
+        && scannedResult.size() > 0) {
+      return getEmptyChunkResult(scannedResult.size());
+    }
+    int currentRow = 0;
+    long[] surrogateResult = null;
+    int noDictionaryColumnIndex = 0;
+    ByteArrayWrapper key = null;
+    Object[] value = null;
+    while (scannedResult.hasNext()) {
+      key = scannedResult.getKey();
+      value = scannedResult.getValue();
+      if (key != null) {
+        surrogateResult = queryExecuterProperties.keyStructureInfo.getKeyGenerator()
+            .getKeyArray(key.getDictionaryKey(),
+                queryExecuterProperties.keyStructureInfo.getMaskedBytes());
+        for (int i = 0; i < dimensionCount; i++) {
+          if (!CarbonUtil.hasEncoding(queryDimension.get(i).getDimension().getEncoder(),
+              Encoding.DICTIONARY)) {
+            resultData[currentRow][i] = DataTypeUtil.getDataBasedOnDataType(
+                new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++),
+                    Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
+                queryDimension.get(i).getDimension().getDataType());
+          } else {
+            resultData[currentRow][i] =
+                (int) surrogateResult[queryDimension.get(i).getDimension().getKeyOrdinal()];
+          }
+        }
+      }
+      if (value != null) {
+        System.arraycopy(value, 0, resultData[currentRow], dimensionCount,
+            queryExecuterProperties.measureDataTypes.length);
+      }
+      currentRow++;
+      noDictionaryColumnIndex = 0;
+    }
+    if (resultData.length > 0) {
+      resultData = encodeToRows(resultData);
+    }
+    return getResult(queryModel, resultData);
+  }
+
+  private BatchResult getResult(QueryModel queryModel, Object[][] convertedResult) {
+
+    int rowSize = convertedResult[0].length;
+    Object[][] rows = new Object[rowSize][];
+    List<QueryDimension> queryDimensions = queryModel.getQueryDimension();
+    int dimensionCount = queryDimensions.size();
+    int msrCount = queryExecuterProperties.measureDataTypes.length;
+    Object[] row;
+    for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) {
+      row = new Object[dimensionCount + msrCount];
+      fillDimensionData(convertedResult, queryDimensions, dimensionCount, row, rowIndex);
+
+      QueryMeasure msr;
+      for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) {
+        msr = queryModel.getQueryMeasures().get(i);
+        row[msr.getQueryOrder()] = convertedResult[dimensionCount + i][rowIndex];
+      }
+      rows[rowIndex] = row;
+    }
+    LOGGER.info(
+        "###########################################------ Total Number of records" + rowSize);
+    BatchResult chunkResult = new BatchResult();
+    chunkResult.setRows(rows);
+    return chunkResult;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java
new file mode 100644
index 0000000..75d78a2
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/result/preparator/impl/RawQueryResultPreparatorImpl.java
@@ -0,0 +1,127 @@
+package org.carbondata.scan.result.preparator.impl;
+
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.scan.executor.impl.QueryExecutorProperties;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.model.QuerySchemaInfo;
+import org.carbondata.scan.result.BatchRawResult;
+import org.carbondata.scan.result.BatchResult;
+import org.carbondata.scan.result.ListBasedResultWrapper;
+import org.carbondata.scan.result.Result;
+import org.carbondata.scan.util.DataTypeUtil;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * It does not decode the dictionary.
+ */
+public class RawQueryResultPreparatorImpl
+    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, Object> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(RawQueryResultPreparatorImpl.class.getName());
+
+  private QuerySchemaInfo querySchemaInfo;
+
+  public RawQueryResultPreparatorImpl(QueryExecutorProperties executerProperties,
+      QueryModel queryModel) {
+    super(executerProperties, queryModel);
+    querySchemaInfo = new QuerySchemaInfo();
+    querySchemaInfo.setKeyGenerator(queryExecuterProperties.keyStructureInfo.getKeyGenerator());
+    querySchemaInfo.setMaskedByteIndexes(queryExecuterProperties.keyStructureInfo.getMaskedBytes());
+    querySchemaInfo.setQueryDimensions(queryModel.getQueryDimension()
+        .toArray(new QueryDimension[queryModel.getQueryDimension().size()]));
+    querySchemaInfo.setQueryMeasures(queryModel.getQueryMeasures()
+        .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
+    int msrSize = queryExecuterProperties.measureDataTypes.length;
+    int dimensionCount = queryModel.getQueryDimension().size();
+    int[] queryOrder = new int[dimensionCount + msrSize];
+    int[] queryReverseOrder = new int[dimensionCount + msrSize];
+    for (int i = 0; i < dimensionCount; i++) {
+      queryOrder[queryModel.getQueryDimension().get(i).getQueryOrder()] = i;
+      queryReverseOrder[i] = queryModel.getQueryDimension().get(i).getQueryOrder();
+    }
+    for (int i = 0; i < msrSize; i++) {
+      queryOrder[queryModel.getQueryMeasures().get(i).getQueryOrder()] = i + dimensionCount;
+      queryReverseOrder[i + dimensionCount] = queryModel.getQueryMeasures().get(i).getQueryOrder();
+    }
+    querySchemaInfo.setQueryOrder(queryOrder);
+    querySchemaInfo.setQueryReverseOrder(queryReverseOrder);
+  }
+
+  @Override public BatchResult prepareQueryResult(
+      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
+    if ((null == scannedResult || scannedResult.size() < 1)) {
+      return new BatchRawResult();
+    }
+    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
+    int msrSize = queryExecuterProperties.measureDataTypes.length;
+    int dimSize = queryDimensions.length;
+    int[] order = querySchemaInfo.getQueryReverseOrder();
+    Object[][] resultData = new Object[scannedResult.size()][];
+    Object[] value;
+    Object[] row;
+    int counter = 0;
+    if (queryModel.isRawBytesDetailQuery()) {
+      while (scannedResult.hasNext()) {
+        value = scannedResult.getValue();
+        row = new Object[msrSize + 1];
+        row[0] = scannedResult.getKey();
+        if (value != null) {
+          assert (value.length == msrSize);
+          System.arraycopy(value, 0, row, 1, msrSize);
+        }
+        resultData[counter] = row;
+        counter++;
+      }
+    } else {
+      while (scannedResult.hasNext()) {
+        value = scannedResult.getValue();
+        row = new Object[msrSize + dimSize];
+        ByteArrayWrapper key = scannedResult.getKey();
+        if (key != null) {
+          long[] surrogateResult = querySchemaInfo.getKeyGenerator()
+              .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
+          int noDictionaryColumnIndex = 0;
+          for (int i = 0; i < dimSize; i++) {
+            if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
+              row[order[i]] = DataTypeUtil.getDataBasedOnDataType(
+                  new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
+                  queryDimensions[i].getDimension().getDataType());
+            } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+              DirectDictionaryGenerator directDictionaryGenerator =
+                  DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
+                      queryDimensions[i].getDimension().getDataType());
+              if (directDictionaryGenerator != null) {
+                row[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
+                    (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
+              }
+            } else {
+              row[order[i]] =
+                  (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
+            }
+          }
+        }
+        for (int i = 0; i < msrSize; i++) {
+          row[order[i + queryDimensions.length]] = value[i];
+        }
+        resultData[counter] = row;
+        counter++;
+      }
+    }
+
+    LOGGER.info("###########################---- Total Number of records" + scannedResult.size());
+    BatchRawResult result = new BatchRawResult();
+    result.setRows(resultData);
+    return result;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/scanner/AbstractBlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/scanner/AbstractBlockletScanner.java b/core/src/main/java/org/carbondata/scan/scanner/AbstractBlockletScanner.java
new file mode 100644
index 0000000..4d7f5d3
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/scanner/AbstractBlockletScanner.java
@@ -0,0 +1,62 @@
+/*
+ * 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.carbondata.scan.scanner;
+
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+import org.carbondata.scan.result.AbstractScannedResult;
+
+/**
+ * Blocklet scanner class to process the block
+ */
+public abstract class AbstractBlockletScanner implements BlockletScanner {
+
+  /**
+   * scanner result
+   */
+  protected AbstractScannedResult scannedResult;
+
+  /**
+   * block execution info
+   */
+  protected BlockExecutionInfo blockExecutionInfo;
+
+  public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) {
+    this.blockExecutionInfo = tableBlockExecutionInfos;
+  }
+
+  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
+      throws QueryExecutionException {
+    fillKeyValue(blocksChunkHolder);
+    return scannedResult;
+  }
+
+  protected void fillKeyValue(BlocksChunkHolder blocksChunkHolder) {
+    scannedResult.reset();
+    scannedResult.setMeasureChunks(blocksChunkHolder.getDataBlock()
+        .getMeasureChunks(blocksChunkHolder.getFileReader(),
+            blockExecutionInfo.getAllSelectedMeasureBlocksIndexes()));
+    scannedResult.setNumberOfRows(blocksChunkHolder.getDataBlock().nodeSize());
+
+    scannedResult.setDimensionChunks(blocksChunkHolder.getDataBlock()
+        .getDimensionChunks(blocksChunkHolder.getFileReader(),
+            blockExecutionInfo.getAllSelectedDimensionBlocksIndexes()));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/scanner/BlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/scanner/BlockletScanner.java b/core/src/main/java/org/carbondata/scan/scanner/BlockletScanner.java
new file mode 100644
index 0000000..f1a0646
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/scanner/BlockletScanner.java
@@ -0,0 +1,41 @@
+/*
+ * 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.carbondata.scan.scanner;
+
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+import org.carbondata.scan.result.AbstractScannedResult;
+
+/**
+ * Interface for processing the block
+ * Processing can be filter based processing or non filter based processing
+ */
+public interface BlockletScanner {
+
+  /**
+   * Below method will used to process the block data and get the scanned result
+   *
+   * @param blocksChunkHolder block chunk which holds the block data
+   * @return scannerResult
+   * result after processing
+   * @throws QueryExecutionException
+   */
+  AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
+      throws QueryExecutionException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/scanner/impl/FilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/scanner/impl/FilterScanner.java b/core/src/main/java/org/carbondata/scan/scanner/impl/FilterScanner.java
new file mode 100644
index 0000000..830146d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/scanner/impl/FilterScanner.java
@@ -0,0 +1,174 @@
+/*
+ * 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.carbondata.scan.scanner.impl;
+
+import java.util.BitSet;
+
+import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.util.CarbonProperties;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.executer.FilterExecuter;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+import org.carbondata.scan.result.AbstractScannedResult;
+import org.carbondata.scan.result.impl.FilterQueryScannedResult;
+import org.carbondata.scan.scanner.AbstractBlockletScanner;
+
+/**
+ * Below class will be used for filter query processing
+ * this class will be first apply the filter then it will read the block if
+ * required and return the scanned result
+ */
+public class FilterScanner extends AbstractBlockletScanner {
+
+  /**
+   * filter tree
+   */
+  private FilterExecuter filterExecuter;
+
+  /**
+   * this will be used to apply min max
+   * this will be useful for dimension column which is on the right side
+   * as node finder will always give tentative blocks, if column data stored individually
+   * and data is in sorted order then we can check whether filter is in the range of min max or not
+   * if it present then only we can apply filter on complete data.
+   * this will be very useful in case of sparse data when rows are
+   * repeating.
+   */
+  private boolean isMinMaxEnabled;
+
+  public FilterScanner(BlockExecutionInfo blockExecutionInfo) {
+    super(blockExecutionInfo);
+    scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
+    // to check whether min max is enabled or not
+    String minMaxEnableValue = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
+            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
+    if (null != minMaxEnableValue) {
+      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
+    }
+    // get the filter tree
+    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
+  }
+
+  /**
+   * Below method will be used to process the block
+   *
+   * @param blocksChunkHolder block chunk holder which holds the data
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
+      throws QueryExecutionException {
+    try {
+      fillScannedResult(blocksChunkHolder);
+    } catch (FilterUnsupportedException e) {
+      throw new QueryExecutionException(e.getMessage());
+    }
+    return scannedResult;
+  }
+
+  /**
+   * This method will process the data in below order
+   * 1. first apply min max on the filter tree and check whether any of the filter
+   * is fall on the range of min max, if not then return empty result
+   * 2. If filter falls on min max range then apply filter on actual
+   * data and get the filtered row index
+   * 3. if row index is empty then return the empty result
+   * 4. if row indexes is not empty then read only those blocks(measure or dimension)
+   * which was present in the query but not present in the filter, as while applying filter
+   * some of the blocks where already read and present in chunk holder so not need to
+   * read those blocks again, this is to avoid reading of same blocks which was already read
+   * 5. Set the blocks and filter indexes to result
+   *
+   * @param blocksChunkHolder
+   * @throws FilterUnsupportedException
+   */
+  private void fillScannedResult(BlocksChunkHolder blocksChunkHolder)
+      throws FilterUnsupportedException {
+
+    scannedResult.reset();
+    // apply min max
+    if (isMinMaxEnabled) {
+      BitSet bitSet = this.filterExecuter
+          .isScanRequired(blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
+              blocksChunkHolder.getDataBlock().getColumnsMinValue());
+      if (bitSet.isEmpty()) {
+        scannedResult.setNumberOfRows(0);
+        scannedResult.setIndexes(new int[0]);
+        return;
+      }
+    }
+    // apply filter on actual data
+    BitSet bitSet = this.filterExecuter.applyFilter(blocksChunkHolder);
+    // if indexes is empty then return with empty result
+    if (bitSet.isEmpty()) {
+      scannedResult.setNumberOfRows(0);
+      scannedResult.setIndexes(new int[0]);
+      return;
+    }
+    // get the row indexes from bot set
+    int[] indexes = new int[bitSet.cardinality()];
+    int index = 0;
+    for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
+      indexes[index++] = i;
+    }
+
+    FileHolder fileReader = blocksChunkHolder.getFileReader();
+    int[] allSelectedDimensionBlocksIndexes =
+        blockExecutionInfo.getAllSelectedDimensionBlocksIndexes();
+    DimensionColumnDataChunk[] dimensionColumnDataChunk =
+        new DimensionColumnDataChunk[blockExecutionInfo.getTotalNumberDimensionBlock()];
+    // read dimension chunk blocks from file which is not present
+    for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) {
+      if (null == blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]]) {
+        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
+            blocksChunkHolder.getDataBlock()
+                .getDimensionChunk(fileReader, allSelectedDimensionBlocksIndexes[i]);
+      } else {
+        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
+            blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]];
+      }
+    }
+    MeasureColumnDataChunk[] measureColumnDataChunk =
+        new MeasureColumnDataChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()];
+    int[] allSelectedMeasureBlocksIndexes = blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
+
+    // read the measure chunk blocks which is not present
+    for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) {
+
+      if (null == blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]]) {
+        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
+            blocksChunkHolder.getDataBlock()
+                .getMeasureChunk(fileReader, allSelectedMeasureBlocksIndexes[i]);
+      } else {
+        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
+            blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]];
+      }
+    }
+    scannedResult.setDimensionChunks(dimensionColumnDataChunk);
+    scannedResult.setIndexes(indexes);
+    scannedResult.setMeasureChunks(measureColumnDataChunk);
+    scannedResult.setNumberOfRows(indexes.length);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/scanner/impl/NonFilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/scanner/impl/NonFilterScanner.java b/core/src/main/java/org/carbondata/scan/scanner/impl/NonFilterScanner.java
new file mode 100644
index 0000000..b582a95
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/scanner/impl/NonFilterScanner.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.carbondata.scan.scanner.impl;
+
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.result.impl.NonFilterQueryScannedResult;
+import org.carbondata.scan.scanner.AbstractBlockletScanner;
+
+/**
+ * Non filter processor which will be used for non filter query
+ * In case of non filter query we just need to read all the blocks requested in the
+ * query and pass it to scanned result
+ */
+public class NonFilterScanner extends AbstractBlockletScanner {
+
+  public NonFilterScanner(BlockExecutionInfo blockExecutionInfo) {
+    super(blockExecutionInfo);
+    // as its a non filter query creating a non filter query scanned result object
+    scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
+  }
+}


[43/56] [abbrv] incubator-carbondata git commit: [Bug] add property CARBON_TIMESTAMP_FORMAT for CarbonExample(#696)

Posted by jb...@apache.org.
[Bug] add property CARBON_TIMESTAMP_FORMAT for CarbonExample(#696)

[Bug] add property CARBON_TIMESTAMP_FORMAT for CarbonExample(#696)

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

Branch: refs/heads/master
Commit: 398250c71645583d8b93b859fa28eaa3591d1f4b
Parents: 91951ac
Author: Jay <li...@huawei.com>
Authored: Mon Jun 20 15:48:13 2016 +0800
Committer: david <Qi...@users.noreply.github.com>
Committed: Mon Jun 20 15:48:13 2016 +0800

----------------------------------------------------------------------
 .../main/scala/org/carbondata/examples/CarbonExample.scala   | 8 ++++++--
 1 file changed, 6 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/398250c7/examples/src/main/scala/org/carbondata/examples/CarbonExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/carbondata/examples/CarbonExample.scala b/examples/src/main/scala/org/carbondata/examples/CarbonExample.scala
index 77907ab..20b2eae 100644
--- a/examples/src/main/scala/org/carbondata/examples/CarbonExample.scala
+++ b/examples/src/main/scala/org/carbondata/examples/CarbonExample.scala
@@ -17,8 +17,8 @@
 
 package org.carbondata.examples
 
-import org.apache.spark.{SparkConf, SparkContext}
-
+import org.carbondata.core.constants.CarbonCommonConstants
+import org.carbondata.core.util.CarbonProperties
 import org.carbondata.examples.util.InitForExamples
 
 object CarbonExample {
@@ -26,6 +26,10 @@ object CarbonExample {
     val cc = InitForExamples.createCarbonContext("CarbonExample")
     val testData = InitForExamples.currentPath + "/src/main/resources/data.csv"
 
+    // Specify timestamp format based on raw data
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/mm/dd")
+
     cc.sql("DROP TABLE IF EXISTS t3")
 
     cc.sql("""


[18/56] [abbrv] incubator-carbondata git commit: Refactored core package and fixed all testcases (#684)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java b/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
deleted file mode 100644
index b46c4de..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
+++ /dev/null
@@ -1,182 +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.carbondata.query.aggregator.util;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.CustomMeasureAggregator;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.avg.AvgBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.avg.AvgDoubleAggregator;
-import org.carbondata.query.aggregator.impl.avg.AvgLongAggregator;
-import org.carbondata.query.aggregator.impl.count.CountAggregator;
-import org.carbondata.query.aggregator.impl.distinct.DistinctCountAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.distinct.DistinctCountBigDecimalAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.distinct.DistinctCountLongAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.distinct.SumDistinctBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.distinct.SumDistinctDoubleAggregator;
-import org.carbondata.query.aggregator.impl.distinct.SumDistinctLongAggregator;
-import org.carbondata.query.aggregator.impl.dummy.DummyBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.dummy.DummyDoubleAggregator;
-import org.carbondata.query.aggregator.impl.dummy.DummyLongAggregator;
-import org.carbondata.query.aggregator.impl.max.MaxAggregator;
-import org.carbondata.query.aggregator.impl.max.MaxBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.max.MaxLongAggregator;
-import org.carbondata.query.aggregator.impl.min.MinAggregator;
-import org.carbondata.query.aggregator.impl.min.MinBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.min.MinLongAggregator;
-import org.carbondata.query.aggregator.impl.sum.SumBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.sum.SumDoubleAggregator;
-import org.carbondata.query.aggregator.impl.sum.SumLongAggregator;
-import org.carbondata.query.carbon.model.CustomAggregateExpression;
-
-/**
- * Factory class to get the measure aggregator
- */
-public class MeasureAggregatorFactory {
-
-  /**
-   * Below method will be used to get the measure aggregator based on type and
-   * and data type
-   *
-   * @param aggTypes                      Aggregation for the column
-   * @param dataTypes                     data type for the column
-   * @param customAggregateExpressionList custom aggregation list to get the
-   *                                      custom aggregation aggregator
-   * @return measure agregator for all the column
-   */
-  public static MeasureAggregator[] getMeassureAggregator(String[] aggTypes, DataType[] dataTypes,
-      List<CustomAggregateExpression> customAggregateExpressionList) {
-    MeasureAggregator[] measureAggregator = new MeasureAggregator[aggTypes.length];
-    int customAggregationCounter = 0;
-    for (int i = 0; i < measureAggregator.length; i++) {
-
-      if (aggTypes[i].equalsIgnoreCase(CarbonCommonConstants.CUSTOM)) {
-        measureAggregator[i] =
-            (CustomMeasureAggregator) customAggregateExpressionList.get(customAggregationCounter++)
-                .getAggregator().getCopy();
-      } else {
-        measureAggregator[i] = getAggregator(aggTypes[i], dataTypes[i]);
-      }
-    }
-    return measureAggregator;
-  }
-
-  /**
-   * Below method will be used to get the aggregate based on aggregator type
-   * and aggregator data type
-   *
-   * @param aggregatorType aggregattor type
-   * @param dataType       data type
-   * @return aggregator
-   */
-  public static MeasureAggregator getAggregator(String aggregatorType, DataType dataType) {
-
-    // get the MeasureAggregator based on aggregate type
-    if (CarbonCommonConstants.MIN.equalsIgnoreCase(aggregatorType)) {
-      switch (dataType) {
-        case LONG:
-          return new MinLongAggregator();
-        case DECIMAL:
-          return new MinBigDecimalAggregator();
-        default:
-          return new MinAggregator();
-      }
-    } else if (CarbonCommonConstants.COUNT.equalsIgnoreCase(aggregatorType)) {
-      return new CountAggregator();
-    }
-    //
-    else if (CarbonCommonConstants.MAX.equalsIgnoreCase(aggregatorType)) {
-      switch (dataType) {
-        case LONG:
-          return new MaxLongAggregator();
-        case DECIMAL:
-          return new MaxBigDecimalAggregator();
-        default:
-          return new MaxAggregator();
-      }
-    }
-    //
-    else if (CarbonCommonConstants.AVERAGE.equalsIgnoreCase(aggregatorType)) {
-      switch (dataType) {
-        case LONG:
-
-          return new AvgLongAggregator();
-        case DECIMAL:
-
-          return new AvgBigDecimalAggregator();
-        default:
-
-          return new AvgDoubleAggregator();
-      }
-    }
-    //
-    else if (CarbonCommonConstants.DISTINCT_COUNT.equalsIgnoreCase(aggregatorType)) {
-      switch (dataType) {
-        case LONG:
-          return new DistinctCountLongAggregatorObjectSet();
-        case DECIMAL:
-          return new DistinctCountBigDecimalAggregatorObjectSet();
-        default:
-          return new DistinctCountAggregatorObjectSet();
-      }
-
-    } else if (CarbonCommonConstants.SUM.equalsIgnoreCase(aggregatorType)) {
-      switch (dataType) {
-        case LONG:
-
-          return new SumLongAggregator();
-        case DECIMAL:
-
-          return new SumBigDecimalAggregator();
-        default:
-
-          return new SumDoubleAggregator();
-      }
-    } else if (CarbonCommonConstants.SUM_DISTINCT.equalsIgnoreCase(aggregatorType)) {
-      switch (dataType) {
-        case LONG:
-
-          return new SumDistinctLongAggregator();
-        case DECIMAL:
-
-          return new SumDistinctBigDecimalAggregator();
-        default:
-
-          return new SumDistinctDoubleAggregator();
-      }
-    } else if (CarbonCommonConstants.DUMMY.equalsIgnoreCase(aggregatorType)) {
-      switch (dataType) {
-        case LONG:
-
-          return new DummyLongAggregator();
-        case DECIMAL:
-
-          return new DummyBigDecimalAggregator();
-        default:
-
-          return new DummyDoubleAggregator();
-      }
-    } else {
-      return null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/DataAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/DataAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/DataAggregator.java
deleted file mode 100644
index efe2fcf..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/DataAggregator.java
+++ /dev/null
@@ -1,76 +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.carbondata.query.carbon.aggregator;
-
-import java.util.List;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.aggregator.expression.ExpressionAggregator;
-import org.carbondata.query.carbon.aggregator.measure.MeasureDataAggregator;
-import org.carbondata.query.carbon.aggregator.measure.impl.FactTableAggregator;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Class which will be used to aggregate all type of aggregation present in the query
- */
-public class DataAggregator {
-
-  /**
-   * measure aggergator which will be used to aggregate the measure columns
-   * preset in the query
-   */
-  private MeasureDataAggregator measureAggregator;
-
-  /**
-   * dimension data aggregator list which will be used to aggregate the
-   * dimension column aggregate function
-   */
-  private List<DimensionDataAggregator> dimensionDataAggergatorList;
-
-  /**
-   * expression aggergator, which will be used to aggregate the expressions
-   * present in the query
-   */
-  private ExpressionAggregator expressionAggregator;
-
-  public DataAggregator(BlockExecutionInfo blockExecutionInfo) {
-    measureAggregator = new FactTableAggregator(blockExecutionInfo);
-    dimensionDataAggergatorList = blockExecutionInfo.getDimensionAggregator();
-    this.expressionAggregator = new ExpressionAggregator(blockExecutionInfo);
-
-  }
-
-  /**
-   * Below method will be used to aggregate the data for all type aggregation
-   * function present in the query Order of aggregation. 1. Dimension column
-   * Aggregation 2. Expression Aggregation 3. Measure column aggregation
-   *
-   * @param scannedResult scanned result
-   * @param aggregators   aggregator
-   */
-  public void aggregateData(AbstractScannedResult scannedResult, MeasureAggregator[] aggregators) {
-    for (int i = 0; i < dimensionDataAggergatorList.size(); i++) {
-      dimensionDataAggergatorList.get(i).aggregateDimensionData(scannedResult, aggregators);
-    }
-    expressionAggregator.aggregateExpression(scannedResult, aggregators);
-    measureAggregator.aggregateMeasure(scannedResult, aggregators);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/DimensionDataAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/DimensionDataAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/DimensionDataAggregator.java
deleted file mode 100644
index 2b58ce4..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/DimensionDataAggregator.java
+++ /dev/null
@@ -1,40 +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.carbondata.query.carbon.aggregator.dimension;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Interface for dimension aggregation This will be used by different type
- * dimensions 1. Fixed length dimensions 2. Variable length dimensions 3.
- * Complex dimensions. 4. Row Group Dimension.
- */
-public interface DimensionDataAggregator {
-
-  /**
-   * Below method will be used to aggregate the dimension data
-   *
-   * @param scannedResult scanned result
-   * @param aggeragtor    aggregator used to aggregate the data
-   */
-  public abstract void aggregateDimensionData(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggeragtor);
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/ColumnGroupDimensionsAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/ColumnGroupDimensionsAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/ColumnGroupDimensionsAggregator.java
deleted file mode 100644
index 0fdc30e..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/ColumnGroupDimensionsAggregator.java
+++ /dev/null
@@ -1,109 +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.carbondata.query.carbon.aggregator.dimension.impl;
-
-import java.util.List;
-
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-
-/**
- * This class will be used to aggregate row group dimension This class will
- * aggregate all the dimensions present in a row group together, as we row group
- * columns data will be stored together so if multiple dimensions of same row
- * group is selected in query, This will be useful when multiple columns of same
- * row group is selected, so unpacking the row group tuple of for the column
- * will be done only once
- */
-public class ColumnGroupDimensionsAggregator implements DimensionDataAggregator {
-
-  /**
-   * info object which store information about dimension to be aggregated
-   */
-  protected List<DimensionAggregatorInfo> dimensionAggeragtorInfo;
-
-  /**
-   * row group key generator which will be used to unpack the row group column
-   * values
-   */
-  private KeyGenerator columnGroupKeyGenerator;
-
-  /**
-   * index of the block in the file for this column group
-   */
-  private int blockIndex;
-
-  /**
-   * dictinoanryInfo;
-   */
-  private List<Dictionary> columnDictionary;
-
-  private int aggregatorStartIndexes;
-
-  public ColumnGroupDimensionsAggregator(List<DimensionAggregatorInfo> dimensionAggeragtorInfo,
-      KeyGenerator columnGroupKeyGenerator, int blockIndex, List<Dictionary> columnDictionary,
-      int aggregatorStartIndexes) {
-
-    this.dimensionAggeragtorInfo = dimensionAggeragtorInfo;
-    this.columnDictionary = columnDictionary;
-    this.columnGroupKeyGenerator = columnGroupKeyGenerator;
-    this.blockIndex = blockIndex;
-    this.aggregatorStartIndexes = aggregatorStartIndexes;
-  }
-
-  /**
-   * Below method will be used to aggregate the dimension data
-   *
-   * @param scannedResult scanned result
-   * @param aggeragtor    aggregator used to aggregate the data
-   */
-  @Override public void aggregateDimensionData(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggeragtor) {
-    long[] surrogateKeyOfColumnGroup = null;
-    surrogateKeyOfColumnGroup =
-        columnGroupKeyGenerator.getKeyArray(scannedResult.getDimensionKey(blockIndex));
-    Object actualData = null;
-    int surrogate = 0;
-    int aggStartIndex = aggregatorStartIndexes;
-    for (int i = 0; i < dimensionAggeragtorInfo.size(); i++) {
-      surrogate = (int) surrogateKeyOfColumnGroup[dimensionAggeragtorInfo.get(i).getDim()
-          .getColumnGroupOrdinal()];
-
-      if (1 == surrogate) {
-        continue;
-      }
-
-      actualData = DataTypeUtil
-          .getDataBasedOnDataType(columnDictionary.get(i).getDictionaryValueForKey(surrogate),
-              dimensionAggeragtorInfo.get(i).getDim().getDataType());
-      if (null == actualData) {
-        continue;
-      }
-      for (int j = 0; j < dimensionAggeragtorInfo.get(i).getAggList().size(); j++) {
-        aggeragtor[aggStartIndex++].agg(actualData);
-      }
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
deleted file mode 100644
index 971e4cc..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
+++ /dev/null
@@ -1,114 +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.carbondata.query.carbon.aggregator.dimension.impl;
-
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Class which will be used to aggregate the direct dictionary dimension data
- */
-public class DirectDictionaryDimensionAggregator implements DimensionDataAggregator {
-
-  /**
-   * info object which store information about dimension to be aggregated
-   */
-  private DimensionAggregatorInfo dimensionAggeragtorInfo;
-
-  /**
-   * start index of the aggregator for current dimension column
-   */
-  private int aggregatorStartIndex;
-
-  /**
-   * buffer used to convert mdkey to surrogate key
-   */
-  private ByteBuffer buffer;
-
-  /**
-   * data index in the file
-   */
-  private int blockIndex;
-
-  /**
-   * to store index which will be used to aggregate
-   * number type value like sum avg
-   */
-  private int[] numberTypeAggregatorIndex;
-
-  /**
-   * DirectDictionaryGenerator
-   */
-  private DirectDictionaryGenerator directDictionaryGenerator;
-
-  /**
-   * to store index which will be used to aggregate
-   * actual type value like max, min, dictinct count
-   */
-  private int[] actualTypeAggregatorIndex;
-
-  public DirectDictionaryDimensionAggregator(DimensionAggregatorInfo dimensionAggeragtorInfo,
-      int aggregatorStartIndex, int blockIndex) {
-    this.dimensionAggeragtorInfo = dimensionAggeragtorInfo;
-    this.aggregatorStartIndex = aggregatorStartIndex;
-    this.blockIndex = blockIndex;
-    buffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
-    numberTypeAggregatorIndex =
-        QueryUtil.getNumberTypeIndex(this.dimensionAggeragtorInfo.getAggList());
-    actualTypeAggregatorIndex =
-        QueryUtil.getActualTypeIndex(this.dimensionAggeragtorInfo.getAggList());
-    directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-        .getDirectDictionaryGenerator(this.dimensionAggeragtorInfo.getDim().getDataType());
-  }
-
-  /**
-   * Below method will be used to aggregate the dimension data
-   *
-   * @param scannedResult scanned result
-   * @param aggeragtor    aggregator used to aggregate the data
-   */
-  @Override public void aggregateDimensionData(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggeragtor) {
-    byte[] dimensionData = scannedResult.getDimensionKey(blockIndex);
-    int surrogateKey = CarbonUtil.getSurrogateKey(dimensionData, buffer);
-    Object dataBasedOnDataType =
-        (long) directDictionaryGenerator.getValueFromSurrogate(surrogateKey) / 1000;
-
-    if (actualTypeAggregatorIndex.length > 0) {
-      for (int j = 0; j < actualTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + actualTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-      }
-    }
-    if (numberTypeAggregatorIndex.length > 0) {
-      for (int j = 0; j < numberTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + numberTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/FixedLengthDimensionAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/FixedLengthDimensionAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/FixedLengthDimensionAggregator.java
deleted file mode 100644
index aaab3e0..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/FixedLengthDimensionAggregator.java
+++ /dev/null
@@ -1,131 +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.carbondata.query.carbon.aggregator.dimension.impl;
-
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-
-/**
- * Class which will be used to aggregate the fixed length dimension data
- */
-public class FixedLengthDimensionAggregator implements DimensionDataAggregator {
-
-  /**
-   * info object which store information about dimension to be aggregated
-   */
-  protected DimensionAggregatorInfo dimensionAggeragtorInfo;
-
-
-  /**
-   * start index of the aggregator for current dimension column
-   */
-  private int aggregatorStartIndex;
-
-  /**
-   * buffer used to convert mdkey to surrogate key
-   */
-  private ByteBuffer buffer;
-
-  /**
-   * dictionary to get the actual value of the column
-   */
-  private Dictionary columnDictionary;
-
-  /**
-   * data index in the file
-   */
-  private int blockIndex;
-
-  /**
-   * to store index which will be used to aggregate
-   * number type value like sum avg
-   */
-  private int[] numberTypeAggregatorIndex;
-
-  /**
-   * to store index which will be used to aggregate
-   * actual type value like max, min, dictinct count
-   */
-  private int[] actualTypeAggregatorIndex;
-
-  public FixedLengthDimensionAggregator(DimensionAggregatorInfo dimensionAggeragtorInfo,
-      Object defaultValue, Dictionary columnDictionary, int aggregatorStartIndex, int blockIndex) {
-    this.dimensionAggeragtorInfo = dimensionAggeragtorInfo;
-    this.aggregatorStartIndex = aggregatorStartIndex;
-    this.blockIndex = blockIndex;
-    buffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
-    this.columnDictionary = columnDictionary;
-    numberTypeAggregatorIndex = QueryUtil.getNumberTypeIndex(dimensionAggeragtorInfo.getAggList());
-    actualTypeAggregatorIndex = QueryUtil.getActualTypeIndex(dimensionAggeragtorInfo.getAggList());
-
-  }
-
-  /**
-   * Below method will be used to aggregate the dimension data
-   *
-   * @param scannedResult scanned result
-   * @param aggeragtor    aggregator used to aggregate the data
-   */
-  @Override public void aggregateDimensionData(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggeragtor) {
-    byte[] dimensionData = scannedResult.getDimensionKey(blockIndex);
-    int surrogateKey = CarbonUtil.getSurrogateKey(dimensionData, buffer);
-    if (surrogateKey == 1) {
-      return;
-    }
-    String actualValue = columnDictionary.getDictionaryValueForKey(surrogateKey);
-    Object dataBasedOnDataType = null;
-    dataBasedOnDataType = DataTypeUtil
-        .getDataBasedOnDataType(actualValue, dimensionAggeragtorInfo.getDim().getDataType());
-    // if data is null then no need to aggregate
-    if (null == dataBasedOnDataType) {
-      return;
-    }
-    if (actualTypeAggregatorIndex.length > 0) {
-      for (int j = 0; j < actualTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + actualTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-      }
-    }
-    // if sum or avg aggregator is applied then first we need to check whether data type
-    // if data type is string then convert to double data type and then apply aggregate
-    // function
-    if (numberTypeAggregatorIndex.length > 0) {
-      if (dimensionAggeragtorInfo.getDim().getDataType().equals(DataType.STRING)) {
-        dataBasedOnDataType = DataTypeUtil.getDataBasedOnDataType(actualValue, DataType.DOUBLE);
-      }
-      if (null == dataBasedOnDataType) {
-        return;
-      }
-      for (int j = 0; j < numberTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + numberTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/VariableLengthDimensionAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/VariableLengthDimensionAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/VariableLengthDimensionAggregator.java
deleted file mode 100644
index f5782b0..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/VariableLengthDimensionAggregator.java
+++ /dev/null
@@ -1,126 +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.carbondata.query.carbon.aggregator.dimension.impl;
-
-import java.nio.charset.Charset;
-
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-
-/**
- * Class which will be used to aggregate the Variable length dimension data
- */
-public class VariableLengthDimensionAggregator implements DimensionDataAggregator {
-
-  /**
-   * info object which store information about dimension to be aggregated
-   */
-  private DimensionAggregatorInfo dimensionAggeragtorInfo;
-
-  /**
-   * default which was added for new dimension after restructuring for the
-   * older blocks
-   */
-  private Object defaultValue;
-
-  /**
-   * index of the aggregator
-   */
-  private int aggregatorStartIndex;
-
-  /**
-   * index of block in file
-   */
-  private int blockIndex;
-
-  /**
-   * to store index which will be used to aggregate
-   * number type value like sum avg
-   */
-  private int[] numberTypeAggregatorIndex;
-
-  /**
-   * to store index which will be used to aggregate
-   * actual type value like max, min, dictinct count
-   */
-  private int[] actualTypeAggregatorIndex;
-
-  public VariableLengthDimensionAggregator(DimensionAggregatorInfo dimensionAggeragtorInfo,
-      Object defaultValue, int aggregatorStartIndex, int blockIndex) {
-    this.dimensionAggeragtorInfo = dimensionAggeragtorInfo;
-    this.defaultValue = defaultValue;
-    this.aggregatorStartIndex = aggregatorStartIndex;
-    this.blockIndex = blockIndex;
-    numberTypeAggregatorIndex = QueryUtil.getNumberTypeIndex(dimensionAggeragtorInfo.getAggList());
-    actualTypeAggregatorIndex = QueryUtil.getActualTypeIndex(dimensionAggeragtorInfo.getAggList());
-
-  }
-
-  /**
-   * Below method will be used to aggregate the variable length dimension data
-   *
-   * @param scannedResult scanned result
-   * @param aggeragtor    aggregator used to aggregate the data
-   */
-  @Override public void aggregateDimensionData(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggeragtor) {
-
-    String data = null;
-    if (defaultValue != null) {
-      data = (String) defaultValue;
-    } else {
-      data = new String(scannedResult.getDimensionKey(blockIndex),
-          Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-      if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(data)) {
-        return;
-      }
-    }
-    Object dataBasedOnDataType =
-        DataTypeUtil.getDataBasedOnDataType(data, dimensionAggeragtorInfo.getDim().getDataType());
-    if (null == dataBasedOnDataType) {
-      return;
-    }
-    if (actualTypeAggregatorIndex.length > 0) {
-      for (int j = 0; j < actualTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + actualTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-      }
-    }
-    // if sum or avg aggregator is applied then first we need to check whether data type
-    // if data type is string then convert to double data type and then apply aggregate
-    // function
-    if (numberTypeAggregatorIndex.length > 0) {
-      if (DataType.STRING==dimensionAggeragtorInfo.getDim().getDataType()) {
-        dataBasedOnDataType = DataTypeUtil.getDataBasedOnDataType(data, DataType.DOUBLE);
-      }
-      if (null == dataBasedOnDataType) {
-        return;
-      }
-      for (int j = 0; j < numberTypeAggregatorIndex.length; j++) {
-        aggeragtor[aggregatorStartIndex + numberTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/expression/ExpressionAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/expression/ExpressionAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/expression/ExpressionAggregator.java
deleted file mode 100644
index b7b79a1..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/expression/ExpressionAggregator.java
+++ /dev/null
@@ -1,135 +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.carbondata.query.carbon.aggregator.expression;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.CustomMeasureAggregator;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbonfilterinterface.RowImpl;
-
-/**
- * Below class will be used for aggregate expression for example if there are
- * two in aggregation f(column1)+f(column2).
- */
-public class ExpressionAggregator {
-
-  /**
-   * aggregator index of the expression
-   */
-  private int expressionStartIndex;
-
-  /**
-   * buffer which will be used to convert the actual dictionary column data to
-   * surrogate key which will be used to get the actual value from dictionary
-   */
-  private ByteBuffer buffer;
-
-  /**
-   * block execution info which has all the detail of expression related
-   * operations
-   */
-  private BlockExecutionInfo blockExecutionInfo;
-
-  public ExpressionAggregator(BlockExecutionInfo blockExecutionInfo) {
-    buffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
-    this.expressionStartIndex =
-        blockExecutionInfo.getAggregatorInfo().getExpressionAggregatorStartIndex();
-    this.blockExecutionInfo = blockExecutionInfo;
-  }
-
-  /**
-   * Below method will be used to aggregate the columns present in expression.
-   * Expression can be on any column dimension and measure so for aggregation
-   * we need to create a row based on and fill the value for all the column
-   * participated in expression and then pass it to aggregator to aggregate
-   * the values
-   *
-   * @param scannedResult     scanned result
-   * @param measureAggregator measure aggregate to aggregate the data
-   */
-  public void aggregateExpression(AbstractScannedResult scannedResult,
-      MeasureAggregator[] measureAggregator) {
-
-    RowImpl rowImpl = null;
-    for (int i = 0; i < this.blockExecutionInfo.getCustomAggregateExpressions().size(); i++) {
-      List<CarbonColumn> referredColumns =
-          this.blockExecutionInfo.getCustomAggregateExpressions().get(i).getReferredColumns();
-      Object[] row = new Object[referredColumns.size()];
-      for (int j = 0; j < referredColumns.size(); j++) {
-        CarbonColumn carbonColumn = referredColumns.get(j);
-        if (!carbonColumn.isDimesion()) {
-          // for measure column first checking whether measure value
-          // was null or not
-          if (!scannedResult.isNullMeasureValue(carbonColumn.getOrdinal())) {
-            // if no null then get the data based on actual data
-            // type
-            switch (carbonColumn.getDataType()) {
-              case LONG:
-                row[j] = scannedResult.getLongMeasureValue(carbonColumn.getOrdinal());
-                break;
-              case DECIMAL:
-                row[j] = scannedResult.getBigDecimalMeasureValue(carbonColumn.getOrdinal());
-                break;
-              default:
-                row[j] = scannedResult.getDoubleMeasureValue(carbonColumn.getOrdinal());
-            }
-          }
-        } else if (!CarbonUtil.hasEncoding(carbonColumn.getEncoder(), Encoding.DICTIONARY)) {
-          // for dictionary column get the data
-          String noDictionaryColumnData =
-              new String(scannedResult.getDimensionKey(carbonColumn.getOrdinal()),
-                  Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          // if data is equal to default value then its null so no
-          // need to do any thing
-          if (!CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(noDictionaryColumnData)) {
-            row[j] = DataTypeUtil
-                .getDataBasedOnDataType(noDictionaryColumnData, carbonColumn.getDataType());
-          }
-        } else {
-
-          // get the surrogate key of the dictionary column from data
-          int surrogateKey = CarbonUtil
-              .getSurrogateKey(scannedResult.getDimensionKey(carbonColumn.getOrdinal()), buffer);
-          // if surrogate key is one then its null value
-          // as we are writing the null value surrogate key as 1
-          if (surrogateKey != 1) {
-            row[j] = DataTypeUtil.getDataBasedOnDataType(
-                blockExecutionInfo.getColumnIdToDcitionaryMapping().get(carbonColumn.getColumnId())
-                    .getDictionaryValueForKey(surrogateKey), carbonColumn.getDataType());
-          }
-        }
-        CustomMeasureAggregator agg =
-            (CustomMeasureAggregator) measureAggregator[expressionStartIndex + i];
-        rowImpl = new RowImpl();
-        rowImpl.setValues(row);
-        agg.agg(rowImpl);
-      }
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
index 8dea48f..074e9ea 100644
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
@@ -23,9 +23,9 @@ import java.util.List;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.DataAggregator;
 import org.carbondata.query.carbon.aggregator.ScannedResultAggregator;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
@@ -34,6 +34,7 @@ import org.carbondata.query.carbon.result.AbstractScannedResult;
 import org.carbondata.query.carbon.result.ListBasedResultWrapper;
 import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.result.impl.ListBasedResult;
+import org.carbondata.query.carbon.util.DataTypeUtil;
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 
 /**
@@ -45,10 +46,6 @@ public class ListBasedResultAggregator implements ScannedResultAggregator {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(ListBasedResultAggregator.class.getName());
-  /**
-   * data aggregator
-   */
-  private DataAggregator dataAggregator;
 
   /**
    * to keep a track of number of row processed to handle limit push down in
@@ -68,14 +65,6 @@ public class ListBasedResultAggregator implements ScannedResultAggregator {
   private List<ListBasedResultWrapper> listBasedResult;
 
   /**
-   * aggreagtor used
-   *
-   * @TODO no need to create this
-   * @TODO need to handle in some other way
-   */
-  private MeasureAggregator[] blockAggregator;
-
-  /**
    * restructuring info
    */
   private KeyStructureInfo restructureInfos;
@@ -85,13 +74,34 @@ public class ListBasedResultAggregator implements ScannedResultAggregator {
    */
   private BlockExecutionInfo tableBlockExecutionInfos;
 
-  public ListBasedResultAggregator(BlockExecutionInfo blockExecutionInfos,
-      DataAggregator aggregator) {
+  private int[] measuresOrdinal;
+
+  /**
+   * to check whether measure exists in current table block or not this to
+   * handle restructuring scenario
+   */
+  private boolean[] isMeasureExistsInCurrentBlock;
+
+  /**
+   * default value of the measures in case of restructuring some measure wont
+   * be present in the table so in that default value will be used to
+   * aggregate the data for that measure columns
+   */
+  private Object[] measureDefaultValue;
+
+  /**
+   * measure datatypes.
+   */
+  private DataType[] measureDatatypes;
+
+  public ListBasedResultAggregator(BlockExecutionInfo blockExecutionInfos) {
     limit = blockExecutionInfos.getLimit();
     this.tableBlockExecutionInfos = blockExecutionInfos;
-    blockAggregator = blockExecutionInfos.getAggregatorInfo().getMeasuresAggreagators();
     restructureInfos = blockExecutionInfos.getKeyStructureInfo();
-    dataAggregator = new DataAggregator(blockExecutionInfos);
+    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
+    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
+    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
+    this.measureDatatypes = tableBlockExecutionInfos.getAggregatorInfo().getMeasureDataTypes();
   }
 
   @Override
@@ -102,52 +112,74 @@ public class ListBasedResultAggregator implements ScannedResultAggregator {
    *
    */
   public int aggregateData(AbstractScannedResult scannedResult) {
-    this.listBasedResult = new ArrayList<ListBasedResultWrapper>(
-        limit == -1 ? scannedResult.numberOfOutputRows() : limit);
+    this.listBasedResult =
+        new ArrayList<>(limit == -1 ? scannedResult.numberOfOutputRows() : limit);
+    boolean isMsrsPresent = measureDatatypes.length > 0;
     ByteArrayWrapper wrapper = null;
-    MeasureAggregator[] measureAggregator = null;
     // scan the record and add to list
     ListBasedResultWrapper resultWrapper;
     while (scannedResult.hasNext() && (limit == -1 || rowCounter < limit)) {
-      wrapper = new ByteArrayWrapper();
-      wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
-      wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
-      wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
       resultWrapper = new ListBasedResultWrapper();
-      measureAggregator=getNewAggregator();
-      resultWrapper.setKey(wrapper);
-      resultWrapper.setValue(measureAggregator);
+      if(tableBlockExecutionInfos.isDimensionsExistInQuery()) {
+        wrapper = new ByteArrayWrapper();
+        wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
+        wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
+        wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
+        resultWrapper.setKey(wrapper);
+      } else {
+        scannedResult.incrementCounter();
+      }
+      if(isMsrsPresent) {
+        Object[] msrValues = new Object[measureDatatypes.length];
+        fillMeasureData(msrValues, scannedResult);
+        resultWrapper.setValue(msrValues);
+      }
       listBasedResult.add(resultWrapper);
       rowCounter++;
-      // call data aggregator to convert measure value to some aggreagtor
-      // object
-      dataAggregator.aggregateData(scannedResult, measureAggregator);
     }
     return rowCounter;
   }
 
-  /**
-   * Below method will be used to get the new aggreagtor
-   *
-   * @return new aggregator object
-   */
-  private MeasureAggregator[] getNewAggregator() {
-    MeasureAggregator[] aggregators = new MeasureAggregator[blockAggregator.length];
-    for (int i = 0; i < blockAggregator.length; i++) {
-      aggregators[i] = blockAggregator[i].getNew();
+  private void fillMeasureData(Object[] msrValues, AbstractScannedResult scannedResult) {
+    for (short i = 0; i < measuresOrdinal.length; i++) {
+      // if measure exists is block then pass measure column
+      // data chunk to the aggregator
+      if (isMeasureExistsInCurrentBlock[i]) {
+        msrValues[i] =
+            getMeasureData(scannedResult.getMeasureChunk(measuresOrdinal[i]),
+                scannedResult.getCurrenrRowId(),measureDatatypes[i]);
+      } else {
+        // if not then get the default value and use that value in aggregation
+        msrValues[i] = measureDefaultValue[i];
+      }
     }
-    return aggregators;
+  }
+
+  private Object getMeasureData(MeasureColumnDataChunk dataChunk, int index, DataType dataType) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      Object msrVal;
+      switch (dataType) {
+        case LONG:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+          break;
+        case DECIMAL:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+          break;
+        default:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+      }
+      return DataTypeUtil.getMeasureDataBasedOnDataType(msrVal, dataType);
+    }
+    return null;
   }
 
   /**
    * Below method will used to get the result
-   *
-   * @param aggregated result
    */
-  @Override public Result<List<ListBasedResultWrapper>> getAggregatedResult() {
-    Result<List<ListBasedResultWrapper>> result = new ListBasedResult();
+  @Override public Result getAggregatedResult() {
+    Result result = new ListBasedResult();
     if (!tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
-      updateKeyWithLatestBlockKeyganerator();
+      updateKeyWithLatestBlockKeygenerator();
       result.addScannedResult(listBasedResult);
     } else {
       result.addScannedResult(listBasedResult);
@@ -155,13 +187,15 @@ public class ListBasedResultAggregator implements ScannedResultAggregator {
     return result;
   }
 
+
+
   /**
    * Below method will be used to update the fixed length key with the
    * latest block key generator
    *
    * @return updated block
    */
-  private void updateKeyWithLatestBlockKeyganerator() {
+  private void updateKeyWithLatestBlockKeygenerator() {
     try {
       long[] data = null;
       ByteArrayWrapper key = null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/MapBasedResultAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/MapBasedResultAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/MapBasedResultAggregator.java
deleted file mode 100644
index 5f4c32f..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/MapBasedResultAggregator.java
+++ /dev/null
@@ -1,171 +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.carbondata.query.carbon.aggregator.impl;
-
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.DataAggregator;
-import org.carbondata.query.carbon.aggregator.ScannedResultAggregator;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.impl.MapBasedResult;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * Scanned result aggregator for aggregated query This will use hash map to
- * aggregate the query result
- */
-public class MapBasedResultAggregator implements ScannedResultAggregator {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(MapBasedResultAggregator.class.getName());
-
-  /**
-   * will be used to aggregate the scaned result
-   */
-  private Map<ByteArrayWrapper, MeasureAggregator[]> aggData;
-
-  /**
-   * interface for data aggregation
-   */
-  private DataAggregator dataAggregator;
-
-  /**
-   * key which will be used for aggregation
-   */
-  private ByteArrayWrapper wrapper;
-
-  /**
-   * aggregate which will be used to get the new aggregator
-   */
-  private MeasureAggregator[] blockAggregator;
-
-  /**
-   * restructuring info
-   */
-  private KeyStructureInfo restructureInfos;
-
-  /**
-   * table block execution infos
-   */
-  private BlockExecutionInfo tableBlockExecutionInfos;
-
-  public MapBasedResultAggregator(BlockExecutionInfo tableBlockExecutionInfos,
-      DataAggregator dataAggregator) {
-    // creating a map of bigger value to avoid rehasing
-    // problem with this is if enough space is not present the memory
-    // this object creation will take more time because of gc
-    aggData = new HashMap<ByteArrayWrapper, MeasureAggregator[]>(100000, 1.0f);
-    this.dataAggregator = dataAggregator;
-    this.wrapper = new ByteArrayWrapper();
-    this.tableBlockExecutionInfos = tableBlockExecutionInfos;
-    blockAggregator = tableBlockExecutionInfos.getAggregatorInfo().getMeasuresAggreagators();
-    restructureInfos = tableBlockExecutionInfos.getKeyStructureInfo();
-  }
-
-  @Override public int aggregateData(AbstractScannedResult scannedResult) {
-
-    while (scannedResult.hasNext()) {
-      // fill the keys
-      wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
-      wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
-      wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
-      MeasureAggregator[] measureAggregators = aggData.get(wrapper);
-      // if null then row was not present in the map
-      // so we need to create a new measure aggregator and
-      // add it to map
-      if (null == measureAggregators) {
-        measureAggregators = getNewAggregator();
-        ByteArrayWrapper byteArrayWrapper = wrapper;
-        wrapper = new ByteArrayWrapper();
-        aggData.put(byteArrayWrapper, measureAggregators);
-      }
-      // aggregate the measure value
-      dataAggregator.aggregateData(scannedResult, measureAggregators);
-    }
-    return 0;
-  }
-
-  /**
-   * Below method will be used to get the new aggreagtor
-   *
-   * @return new aggregator object
-   */
-  private MeasureAggregator[] getNewAggregator() {
-    MeasureAggregator[] aggregators = new MeasureAggregator[blockAggregator.length];
-    for (int i = 0; i < blockAggregator.length; i++) {
-      aggregators[i] = blockAggregator[i].getNew();
-    }
-    return aggregators;
-  }
-
-  /**
-   * Below method will used to get the result
-   *
-   * @param aggregated result
-   */
-  @Override public Result<Map<ByteArrayWrapper, MeasureAggregator[]>> getAggregatedResult() {
-    Result<Map<ByteArrayWrapper, MeasureAggregator[]>> result = new MapBasedResult();
-    updateAggregatedResult();
-    result.addScannedResult(aggData);
-    return result;
-  }
-
-  /**
-   * Below method will be used to update the fixed length key with the latest
-   * block key generator
-   *
-   * @return updated block
-   */
-  private void updateAggregatedResult() {
-    if (!tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
-      return;
-    }
-    try {
-      long[] data = null;
-      ByteArrayWrapper key = null;
-      for (Entry<ByteArrayWrapper, MeasureAggregator[]> e : aggData.entrySet()) {
-        // get the key
-        key = e.getKey();
-        // unpack the key with table block key generator
-        data = tableBlockExecutionInfos.getBlockKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(), tableBlockExecutionInfos.getMaskedByteForBlock());
-        // packed the key with latest block key generator
-        // and generate the masked key for that key
-        key.setDictionaryKey(QueryUtil
-            .getMaskedKey(restructureInfos.getKeyGenerator().generateKey(data),
-                restructureInfos.getMaxKey(), restructureInfos.getMaskByteRanges(),
-                restructureInfos.getMaskByteRanges().length));
-        aggData.put(key, e.getValue());
-      }
-    } catch (KeyGenException e) {
-      LOGGER.error(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/MeasureDataAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/MeasureDataAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/MeasureDataAggregator.java
deleted file mode 100644
index ac03943..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/MeasureDataAggregator.java
+++ /dev/null
@@ -1,48 +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.carbondata.query.carbon.aggregator.measure;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Interface for measure aggregation
- */
-public abstract class MeasureDataAggregator {
-
-  /**
-   * measure ordinal selected in query
-   */
-  protected int[] measuresOrdinal;
-
-  public MeasureDataAggregator(BlockExecutionInfo tableBlockExecutionInfos) {
-    // get the measure ordinal
-    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
-  }
-
-  /**
-   * Below method will be used to aggregate the measures
-   *
-   * @param scannedRsult scanned result
-   * @param aggrgeator   aggregator selected for each measure
-   */
-  public abstract void aggregateMeasure(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggrgeator);
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/impl/FactTableAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/impl/FactTableAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/impl/FactTableAggregator.java
deleted file mode 100644
index 4c2b200..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/measure/impl/FactTableAggregator.java
+++ /dev/null
@@ -1,87 +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.carbondata.query.carbon.aggregator.measure.impl;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.aggregator.measure.MeasureDataAggregator;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Below class will be used to aggregate the data.
- * This class will be used only for fact table.
- * as aggregate table aggregation logic will be different
- * For example:
- * Restructuring is only supported in fact table
- * In case of aggregate table some of the measure columns
- * will be stored in byte array, this based on the aggregation function
- * applied on that measure
- */
-public class FactTableAggregator extends MeasureDataAggregator {
-
-  /**
-   * to check whether measure exists in current table block or not this to
-   * handle restructuring scenario
-   */
-  private boolean[] isMeasureExistsInCurrentBlock;
-
-  /**
-   * default value of the measures in case of restructuring some measure wont
-   * be present in the table so in that default value will be used to
-   * aggregate the data for that measure columns
-   */
-  private Object[] measureDefaultValue;
-
-  /**
-   * as measure column will be aggregated at last so this index will be used
-   * to get the aggergator
-   */
-  private int measureColumnStartIndex;
-
-  public FactTableAggregator(BlockExecutionInfo tableBlockExecutionInfos) {
-    super(tableBlockExecutionInfos);
-    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
-    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
-    measureColumnStartIndex =
-        tableBlockExecutionInfos.getAggregatorInfo().getMeasureAggregatorStartIndex();
-  }
-
-  /**
-   * Below method will be used to aggregate the measure value
-   *
-   * @param scannedRsult scanned result;
-   * @param aggrgeator   aggregator for aggregation
-   */
-  @Override public void aggregateMeasure(AbstractScannedResult scannedResult,
-      MeasureAggregator[] aggrgeator) {
-    for (short i = 0; i < measuresOrdinal.length; i++) {
-      // if measure exists is block then pass measure column
-      // data chunk to the aggregator
-      if (isMeasureExistsInCurrentBlock[i]) {
-        aggrgeator[measureColumnStartIndex + i]
-            .agg(scannedResult.getMeasureChunk(measuresOrdinal[i]),
-                scannedResult.getCurrenrRowId());
-      } else {
-        // if not then get the default value and use that value in aggregation
-        aggrgeator[measureColumnStartIndex + i].agg(measureDefaultValue[i]);
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java b/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java
index 01ff5f1..fd1cf40 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java
@@ -18,14 +18,8 @@
  */
 package org.carbondata.query.carbon.executor;
 
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.query.carbon.executor.impl.AggregationQueryExecutor;
-import org.carbondata.query.carbon.executor.impl.CountStarQueryExecutor;
 import org.carbondata.query.carbon.executor.impl.DetailQueryExecutor;
-import org.carbondata.query.carbon.executor.impl.DetailRawRecordQueryExcecutor;
-import org.carbondata.query.carbon.executor.impl.DetailWithOrderByQueryExecutor;
-import org.carbondata.query.carbon.executor.impl.FunctionQueryExecutor;
+import org.carbondata.query.carbon.executor.impl.DetailRawRecordQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
 
 /**
@@ -34,46 +28,10 @@ import org.carbondata.query.carbon.model.QueryModel;
  */
 public class QueryExecutorFactory {
 
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(QueryExecutorFactory.class.getName());
-
   public static QueryExecutor getQueryExecutor(QueryModel queryModel) {
-    // if all the other query property like query dimension dimension
-    // aggregation expression are empty and is count start query is true
-    // with one measure on which counter will be executed
-    // then its a counter start query
-    if (queryModel.isCountStarQuery() && null == queryModel.getFilterExpressionResolverTree()
-        && queryModel.getQueryDimension().size() < 1 && queryModel.getQueryMeasures().size() < 2
-        && queryModel.getDimAggregationInfo().size() < 1
-        && queryModel.getExpressions().size() == 0 && !queryModel.isForcedDetailRawQuery()) {
-      LOGGER.info("Count(*) query: ");
-      return new CountStarQueryExecutor();
-    }
-    // if all the query property is empty then is a function query like
-    // count(1)
-    // in that case we need to return empty record of size number of records
-    // present in the carbon data file
-    else if (null == queryModel.getFilterExpressionResolverTree()
-        && queryModel.getQueryDimension().size() == 0 && queryModel.getQueryMeasures().size() == 0
-        && queryModel.getDimAggregationInfo().size() == 0
-        && queryModel.getExpressions().size() == 0 && !queryModel.isForcedDetailRawQuery()) {
-      LOGGER.info("Function query: ");
-      return new FunctionQueryExecutor();
-    }
-    // if not a detail query then it is a aggregation query
-    else if (!queryModel.isDetailQuery() && !queryModel.isForcedDetailRawQuery()) {
-      LOGGER.info("Aggergation query: ");
-      return new AggregationQueryExecutor();
-    }
-    // to handle detail with order by query
-    else if (queryModel.isDetailQuery() && queryModel.getSortDimension().size() > 0) {
-      LOGGER.info("Detail with order by query: ");
-      return new DetailWithOrderByQueryExecutor();
-    } else if (queryModel.isForcedDetailRawQuery()) {
-      return new DetailRawRecordQueryExcecutor();
+    if (queryModel.isForcedDetailRawQuery()) {
+      return new DetailRawRecordQueryExecutor();
     } else {
-      // detail query
-      LOGGER.info("Detail query: ");
       return new DetailQueryExecutor();
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java
index 52b3e70..37c4b68 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java
@@ -21,7 +21,6 @@ package org.carbondata.query.carbon.executor.impl;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
-import java.util.Iterator;
 import java.util.List;
 import java.util.Set;
 
@@ -42,7 +41,6 @@ import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.util.MeasureAggregatorFactory;
 import org.carbondata.query.carbon.executor.QueryExecutor;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.AggregatorInfo;
@@ -51,7 +49,6 @@ import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
 import org.carbondata.query.carbon.executor.infos.SortInfo;
 import org.carbondata.query.carbon.executor.util.QueryUtil;
 import org.carbondata.query.carbon.executor.util.RestructureUtil;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
 import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QueryMeasure;
 import org.carbondata.query.carbon.model.QueryModel;
@@ -108,34 +105,12 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
             .getDimensionKeyGenerator());
 
     // calculating the total number of aggeragted columns
-    int aggTypeCount = queryModel.getQueryMeasures().size() + queryModel.getExpressions().size();
+    int aggTypeCount = queryModel.getQueryMeasures().size();
 
-    // as in one dimension multiple aggregator can be selected , so we need
-    // to select all the aggregator function
-    Iterator<DimensionAggregatorInfo> iterator = queryModel.getDimAggregationInfo().iterator();
-    while (iterator.hasNext()) {
-      aggTypeCount += iterator.next().getAggList().size();
-    }
     int currentIndex = 0;
     String[] aggTypes = new String[aggTypeCount];
     DataType[] dataTypes = new DataType[aggTypeCount];
 
-    // adding query dimension selected in aggregation info
-    for (DimensionAggregatorInfo dimensionAggregationInfo : queryModel.getDimAggregationInfo()) {
-      for (int i = 0; i < dimensionAggregationInfo.getAggList().size(); i++) {
-        aggTypes[currentIndex] = dimensionAggregationInfo.getAggList().get(i);
-        dataTypes[currentIndex] = dimensionAggregationInfo.getDim().getDataType();
-        currentIndex++;
-      }
-    }
-    // filling the query expression data type and its aggregator
-    // in this case both will be custom
-    for (int i = 0; i < queryModel.getExpressions().size(); i++) {
-      aggTypes[currentIndex] = CarbonCommonConstants.CUSTOM;
-      dataTypes[currentIndex] = DataType.STRING;
-      currentIndex++;
-    }
-
     for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
       // adding the data type and aggregation type of all the measure this
       // can be used
@@ -144,23 +119,20 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
       dataTypes[currentIndex] = carbonMeasure.getMeasure().getDataType();
       currentIndex++;
     }
-    queryProperties.measureAggregators = MeasureAggregatorFactory
-        .getMeassureAggregator(aggTypes, dataTypes, queryModel.getExpressions());
+    queryProperties.measureDataTypes = dataTypes;
     // as aggregation will be executed in following order
     // 1.aggregate dimension expression
     // 2. expression
     // 3. query measure
     // so calculating the index of the expression start index
     // and measure column start index
-    queryProperties.aggExpressionStartIndex =
-        aggTypes.length - queryModel.getExpressions().size() - queryModel.getQueryMeasures().size();
+    queryProperties.aggExpressionStartIndex = queryModel.getQueryMeasures().size();
     queryProperties.measureStartIndex = aggTypes.length - queryModel.getQueryMeasures().size();
 
     // dictionary column unique column id to dictionary mapping
     // which will be used to get column actual data
     queryProperties.columnToDictionayMapping = QueryUtil
         .getDimensionDictionaryDetail(queryModel.getQueryDimension(),
-            queryModel.getDimAggregationInfo(), queryModel.getExpressions(),
             queryModel.getAbsoluteTableIdentifier());
     queryModel.setColumnToDictionaryMapping(queryProperties.columnToDictionayMapping);
     // setting the sort dimension index. as it will be updated while getting the sort info
@@ -240,12 +212,11 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
         QueryUtil.getMaskedByteRange(updatedQueryDimension, blockKeyGenerator);
     int[] maksedByte =
         QueryUtil.getMaskedByte(blockKeyGenerator.getKeySizeInBytes(), maskByteRangesForBlock);
+    blockExecutionInfo.setDimensionsExistInQuery(updatedQueryDimension.size() > 0);
     blockExecutionInfo.setDataBlock(blockIndex);
     blockExecutionInfo.setBlockKeyGenerator(blockKeyGenerator);
     // adding aggregation info for query
     blockExecutionInfo.setAggregatorInfo(getAggregatorInfoForBlock(queryModel, blockIndex));
-    // adding custom aggregate expression of query
-    blockExecutionInfo.setCustomAggregateExpressions(queryModel.getExpressions());
 
     // setting the limit
     blockExecutionInfo.setLimit(queryModel.getLimit());
@@ -296,15 +267,10 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     // expression measure
     List<CarbonMeasure> expressionMeasures =
         new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // to get all the dimension and measure which required to get the chunk
-    // indexes to be read from file
-    QueryUtil.extractDimensionsAndMeasuresFromExpression(queryModel.getExpressions(),
-        expressionDimensions, expressionMeasures);
     // setting all the dimension chunk indexes to be read from file
     blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(QueryUtil
         .getDimensionsBlockIndexes(updatedQueryDimension,
-            segmentProperties.getDimensionOrdinalToBlockMapping(),
-            queryModel.getDimAggregationInfo(), expressionDimensions));
+            segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions));
     // setting all the measure chunk indexes to be read from file
     blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(QueryUtil
         .getMeasureBlockIndexes(queryModel.getQueryMeasures(), expressionMeasures,
@@ -334,11 +300,6 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     blockExecutionInfo.setColumnIdToDcitionaryMapping(queryProperties.columnToDictionayMapping);
     // setting each column value size
     blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize());
-    blockExecutionInfo.setDimensionAggregator(QueryUtil
-        .getDimensionDataAggregatorList1(queryModel.getDimAggregationInfo(),
-            segmentProperties.getDimensionOrdinalToBlockMapping(),
-            segmentProperties.getColumnGroupAndItsKeygenartor(),
-            queryProperties.columnToDictionayMapping));
     try {
       // to set column group and its key structure info which will be used
       // to
@@ -435,7 +396,7 @@ public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
     aggregatorInfos.setMeasureAggregatorStartIndex(queryProperties.measureStartIndex);
     // setting the measure aggregator for all aggregation function selected
     // in query
-    aggregatorInfos.setMeasuresAggreagators(queryProperties.measureAggregators);
+    aggregatorInfos.setMeasureDataTypes(queryProperties.measureDataTypes);
     return aggregatorInfos;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/AggregationQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/AggregationQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/AggregationQueryExecutor.java
deleted file mode 100644
index d93275e..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/AggregationQueryExecutor.java
+++ /dev/null
@@ -1,47 +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.carbondata.query.carbon.executor.impl;
-
-import java.util.List;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.executor.internal.impl.InternalAggregationQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.RowResult;
-import org.carbondata.query.carbon.result.iterator.ChunkBasedResultIterator;
-import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
-
-/**
- * Below class will be used to execute the aggregation query
- */
-public class AggregationQueryExecutor extends AbstractQueryExecutor<RowResult> {
-
-  @Override public CarbonIterator<RowResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    InternalQueryExecutor internalQueryExecutor = new InternalAggregationQueryExecutor();
-    return new ChunkRowIterator(new ChunkBasedResultIterator(
-        internalQueryExecutor.executeQuery(blockExecutionInfoList, null), queryProperties,
-        queryModel));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/CountStarQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/CountStarQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/CountStarQueryExecutor.java
deleted file mode 100644
index 2542d1d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/CountStarQueryExecutor.java
+++ /dev/null
@@ -1,45 +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.carbondata.query.carbon.executor.impl;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.executor.internal.impl.InternalCountStartQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.RowResult;
-import org.carbondata.query.carbon.result.iterator.ChunkBasedResultIterator;
-import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
-
-/**
- * Below class will be used to execute the count start query
- */
-public class CountStarQueryExecutor extends AbstractQueryExecutor<RowResult> {
-
-  @Override public CarbonIterator<RowResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    initQuery(queryModel);
-    InternalQueryExecutor queryExecutor =
-        new InternalCountStartQueryExecutor(queryProperties.dataBlocks);
-    return new ChunkRowIterator(
-        new ChunkBasedResultIterator(queryExecutor.executeQuery(null, null), queryProperties,
-            queryModel));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
index 371ccc0..b2f323c 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
@@ -26,7 +26,6 @@ import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.executor.internal.impl.InternalDetailQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.RowResult;
 import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
 import org.carbondata.query.carbon.result.iterator.DetailQueryResultIterator;
 
@@ -35,9 +34,9 @@ import org.carbondata.query.carbon.result.iterator.DetailQueryResultIterator;
  * For executing the detail query it will pass all the block execution
  * info to detail query result iterator and iterator will be returned
  */
-public class DetailQueryExecutor extends AbstractQueryExecutor<RowResult> {
+public class DetailQueryExecutor extends AbstractQueryExecutor {
 
-  @Override public CarbonIterator<RowResult> execute(QueryModel queryModel)
+  @Override public CarbonIterator<Object[]> execute(QueryModel queryModel)
       throws QueryExecutionException {
     List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
     InternalQueryExecutor queryExecutor = new InternalDetailQueryExecutor();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExcecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExcecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExcecutor.java
deleted file mode 100644
index d4452da..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExcecutor.java
+++ /dev/null
@@ -1,26 +0,0 @@
-package org.carbondata.query.carbon.executor.impl;
-
-import java.util.List;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.executor.internal.impl.InternalDetailQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchRawResult;
-import org.carbondata.query.carbon.result.iterator.DetailRawQueryResultIterator;
-
-/**
- * Executor for raw records, it does not parse to actual data
- */
-public class DetailRawRecordQueryExcecutor extends AbstractQueryExecutor<BatchRawResult> {
-
-  @Override public CarbonIterator<BatchRawResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    InternalQueryExecutor queryExecutor = new InternalDetailQueryExecutor();
-    return new DetailRawQueryResultIterator(blockExecutionInfoList, queryProperties, queryModel,
-        queryExecutor);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
new file mode 100644
index 0000000..d1967cd
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
@@ -0,0 +1,26 @@
+package org.carbondata.query.carbon.executor.impl;
+
+import java.util.List;
+
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
+import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
+import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
+import org.carbondata.query.carbon.executor.internal.impl.InternalDetailQueryExecutor;
+import org.carbondata.query.carbon.model.QueryModel;
+import org.carbondata.query.carbon.result.BatchResult;
+import org.carbondata.query.carbon.result.iterator.DetailRawQueryResultIterator;
+
+/**
+ * Executor for raw records, it does not parse to actual data
+ */
+public class DetailRawRecordQueryExecutor extends AbstractQueryExecutor<BatchResult> {
+
+  @Override public CarbonIterator<BatchResult> execute(QueryModel queryModel)
+      throws QueryExecutionException {
+    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
+    InternalQueryExecutor queryExecutor = new InternalDetailQueryExecutor();
+    return new DetailRawQueryResultIterator(blockExecutionInfoList, queryProperties, queryModel,
+        queryExecutor);
+  }
+}


[33/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java b/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java
deleted file mode 100644
index eea27e9..0000000
--- a/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java
+++ /dev/null
@@ -1,361 +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.carbondata.query.filters;
-
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.carbon.datastore.DataRefNodeFinder;
-import org.carbondata.core.carbon.datastore.IndexKey;
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.expression.BinaryExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.expression.logical.BinaryLogicalExpression;
-import org.carbondata.query.filter.executer.FilterExecuter;
-import org.carbondata.query.filter.resolver.ConditionalFilterResolverImpl;
-import org.carbondata.query.filter.resolver.FilterResolverIntf;
-import org.carbondata.query.filter.resolver.LogicalFilterResolverImpl;
-import org.carbondata.query.filter.resolver.RowLevelFilterResolverImpl;
-import org.carbondata.query.filter.resolver.RowLevelRangeFilterResolverImpl;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class FilterExpressionProcessor implements FilterProcessor {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(FilterExpressionProcessor.class.getName());
-
-  /**
-   * Implementation will provide the resolved form of filters based on the
-   * filter expression tree which is been passed in Expression instance.
-   *
-   * @param expressionTree  , filter expression tree
-   * @param tableIdentifier ,contains carbon store informations
-   * @return a filter resolver tree
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  public FilterResolverIntf getFilterResolver(Expression expressionTree,
-      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException {
-    if (null != expressionTree && null != tableIdentifier) {
-      return getFilterResolvertree(expressionTree, tableIdentifier);
-    }
-    return null;
-  }
-
-  /**
-   * This API will scan the Segment level all btrees and selects the required
-   * block reference  nodes inorder to push the same to executer for applying filters
-   * on the respective data reference node.
-   * Following Algorithm is followed in below API
-   * Step:1 Get the start end key based on the filter tree resolver information
-   * Step:2 Prepare the IndexKeys inorder to scan the tree and get the start and end reference
-   * node(block)
-   * Step:3 Once data reference node ranges retrieved traverse the node within this range
-   * and select the node based on the block min and max value and the filter value.
-   * Step:4 The selected blocks will be send to executers for applying the filters with the help
-   * of Filter executers.
-   *
-   * @throws QueryExecutionException
-   */
-  public List<DataRefNode> getFilterredBlocks(DataRefNode btreeNode,
-      FilterResolverIntf filterResolver, AbstractIndex tableSegment,
-      AbsoluteTableIdentifier tableIdentifier) throws QueryExecutionException {
-    // Need to get the current dimension tables
-    List<DataRefNode> listOfDataBlocksToScan = new ArrayList<DataRefNode>();
-    // getting the start and end index key based on filter for hitting the
-    // selected block reference nodes based on filter resolver tree.
-    LOGGER.info("preparing the start and end key for finding"
-        + "start and end block as per filter resolver");
-    List<IndexKey> listOfStartEndKeys = new ArrayList<IndexKey>(2);
-    FilterUtil.traverseResolverTreeAndGetStartAndEndKey(tableSegment.getSegmentProperties(),
-        tableIdentifier, filterResolver, listOfStartEndKeys);
-    // reading the first value from list which has start key
-    IndexKey searchStartKey = listOfStartEndKeys.get(0);
-    // reading the last value from list which has end key
-    IndexKey searchEndKey = listOfStartEndKeys.get(1);
-    if (null == searchStartKey && null == searchEndKey) {
-      try {
-        // TODO need to handle for no dictionary dimensions
-        searchStartKey =
-            FilterUtil.prepareDefaultStartIndexKey(tableSegment.getSegmentProperties());
-        // TODO need to handle for no dictionary dimensions
-        searchEndKey = FilterUtil.prepareDefaultEndIndexKey(tableSegment.getSegmentProperties());
-      } catch (KeyGenException e) {
-        return listOfDataBlocksToScan;
-      }
-    }
-
-    LOGGER.info("Successfully retrieved the start and end key");
-    long startTimeInMillis = System.currentTimeMillis();
-    DataRefNodeFinder blockFinder = new BTreeDataRefNodeFinder(
-        tableSegment.getSegmentProperties().getDimensionColumnsValueSize());
-    DataRefNode startBlock = blockFinder.findFirstDataBlock(btreeNode, searchStartKey);
-    DataRefNode endBlock = blockFinder.findLastDataBlock(btreeNode, searchEndKey);
-    FilterExecuter filterExecuter =
-            FilterUtil.getFilterExecuterTree(filterResolver, tableSegment.getSegmentProperties());
-    while (startBlock != endBlock) {
-      addBlockBasedOnMinMaxValue(filterExecuter, listOfDataBlocksToScan, startBlock,
-          tableSegment.getSegmentProperties());
-      startBlock = startBlock.getNextDataRefNode();
-    }
-    addBlockBasedOnMinMaxValue(filterExecuter, listOfDataBlocksToScan, endBlock,
-        tableSegment.getSegmentProperties());
-    LOGGER.info("Total Time in retrieving the data reference node" + "after scanning the btree " + (
-        System.currentTimeMillis() - startTimeInMillis)
-        + " Total number of data reference node for executing filter(s) " + listOfDataBlocksToScan
-        .size());
-
-    return listOfDataBlocksToScan;
-  }
-
-  /**
-   * Selects the blocks based on col max and min value.
-   *
-   * @param filterResolver
-   * @param listOfDataBlocksToScan
-   * @param dataRefNode
-   * @param segmentProperties
-   */
-  private void addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter,
-      List<DataRefNode> listOfDataBlocksToScan, DataRefNode dataRefNode,
-      SegmentProperties segmentProperties) {
-
-    BitSet bitSet = filterExecuter
-        .isScanRequired(dataRefNode.getColumnsMaxValue(), dataRefNode.getColumnsMinValue());
-    if (!bitSet.isEmpty()) {
-      listOfDataBlocksToScan.add(dataRefNode);
-
-    }
-  }
-
-  /**
-   * API will return a filter resolver instance which will be used by
-   * executers to evaluate or execute the filters.
-   *
-   * @param expressionTree , resolver tree which will hold the resolver tree based on
-   *                       filter expression.
-   * @return FilterResolverIntf type.
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  private FilterResolverIntf getFilterResolvertree(Expression expressionTree,
-      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException {
-    FilterResolverIntf filterEvaluatorTree =
-        createFilterResolverTree(expressionTree, tableIdentifier, null);
-    traverseAndResolveTree(filterEvaluatorTree, tableIdentifier);
-    return filterEvaluatorTree;
-  }
-
-  /**
-   * constructing the filter resolver tree based on filter expression.
-   * this method will visit each node of the filter resolver and prepares
-   * the surrogates of the filter members which are involved filter
-   * expression.
-   *
-   * @param filterResolverTree
-   * @param tableIdentifier
-   * @throws QueryExecutionException
-   */
-  private void traverseAndResolveTree(FilterResolverIntf filterResolverTree,
-      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException {
-    if (null == filterResolverTree) {
-      return;
-    }
-    traverseAndResolveTree(filterResolverTree.getLeft(), tableIdentifier);
-
-    filterResolverTree.resolve(tableIdentifier);
-
-    traverseAndResolveTree(filterResolverTree.getRight(), tableIdentifier);
-  }
-
-  /**
-   * Pattern used : Visitor Pattern
-   * Method will create filter resolver tree based on the filter expression tree,
-   * in this algorithm based on the expression instance the resolvers will created
-   *
-   * @param expressionTree
-   * @param tableIdentifier
-   * @return
-   */
-  private FilterResolverIntf createFilterResolverTree(Expression expressionTree,
-      AbsoluteTableIdentifier tableIdentifier, Expression intermediateExpression) {
-    ExpressionType filterExpressionType = expressionTree.getFilterExpressionType();
-    BinaryExpression currentExpression = null;
-    BinaryLogicalExpression logicalExpression = null;
-    switch (filterExpressionType) {
-      case OR:
-        currentExpression = (BinaryExpression) expressionTree;
-        return new LogicalFilterResolverImpl(
-            createFilterResolverTree(currentExpression.getLeft(), tableIdentifier,
-                currentExpression),
-            createFilterResolverTree(currentExpression.getRight(), tableIdentifier,
-                currentExpression), filterExpressionType);
-      case AND:
-        logicalExpression = (BinaryLogicalExpression) expressionTree;
-        return new LogicalFilterResolverImpl(
-            createFilterResolverTree(logicalExpression.getLeft(), tableIdentifier,
-                currentExpression),
-            createFilterResolverTree(logicalExpression.getRight(), tableIdentifier,
-                currentExpression), filterExpressionType);
-      case EQUALS:
-      case IN:
-        return getFilterResolverBasedOnExpressionType(ExpressionType.EQUALS, false, expressionTree,
-            tableIdentifier, expressionTree);
-      case GREATERTHAN:
-      case GREATERTHAN_EQUALTO:
-      case LESSTHAN:
-      case LESSTHAN_EQUALTO:
-        return getFilterResolverBasedOnExpressionType(ExpressionType.EQUALS, true, expressionTree,
-            tableIdentifier, expressionTree);
-
-      case NOT_EQUALS:
-      case NOT_IN:
-        return getFilterResolverBasedOnExpressionType(ExpressionType.NOT_EQUALS, false,
-            expressionTree, tableIdentifier, expressionTree);
-
-      default:
-        return getFilterResolverBasedOnExpressionType(ExpressionType.UNKNOWN, false, expressionTree,
-            tableIdentifier, expressionTree);
-    }
-  }
-
-  /**
-   * Factory method which will return the resolver instance based on filter expression
-   * expressions.
-   */
-  private FilterResolverIntf getFilterResolverBasedOnExpressionType(
-      ExpressionType filterExpressionType, boolean isExpressionResolve, Expression expression,
-      AbsoluteTableIdentifier tableIdentifier, Expression expressionTree) {
-    BinaryConditionalExpression currentCondExpression = null;
-    ConditionalExpression condExpression = null;
-    switch (filterExpressionType) {
-      case EQUALS:
-        currentCondExpression = (BinaryConditionalExpression) expression;
-        if (currentCondExpression.isSingleDimension()
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.ARRAY
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.STRUCT) {
-          // getting new dim index.
-          if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
-              .hasEncoding(Encoding.DICTIONARY) || currentCondExpression.getColumnList().get(0)
-              .getCarbonColumn().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-            if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
-                && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight()) || (
-                FilterUtil.checkIfExpressionContainsUnknownExp(currentCondExpression.getRight())
-                    || FilterUtil
-                    .checkIfExpressionContainsUnknownExp(currentCondExpression.getLeft()))) {
-              return new RowLevelFilterResolverImpl(expression, isExpressionResolve, true,
-                  tableIdentifier);
-            }
-            if (currentCondExpression.getFilterExpressionType() == ExpressionType.GREATERTHAN
-                || currentCondExpression.getFilterExpressionType() == ExpressionType.LESSTHAN
-                || currentCondExpression.getFilterExpressionType()
-                == ExpressionType.GREATERTHAN_EQUALTO
-                || currentCondExpression.getFilterExpressionType()
-                == ExpressionType.LESSTHAN_EQUALTO) {
-              if (currentCondExpression.getColumnList().get(0).getCarbonColumn()
-                  .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-                return new RowLevelFilterResolverImpl(expression, isExpressionResolve, true,
-                    tableIdentifier);
-              }
-              return new RowLevelRangeFilterResolverImpl(expression, isExpressionResolve, true,
-                  tableIdentifier);
-            }
-          }
-          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true);
-
-        }
-        break;
-      case NOT_EQUALS:
-        currentCondExpression = (BinaryConditionalExpression) expression;
-        if (currentCondExpression.isSingleDimension()
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.ARRAY
-            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.STRUCT) {
-          if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
-              .hasEncoding(Encoding.DICTIONARY)) {
-            if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
-                && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight())
-                || (FilterUtil
-                    .checkIfExpressionContainsUnknownExp(currentCondExpression.getRight())
-                || FilterUtil
-                    .checkIfExpressionContainsUnknownExp(currentCondExpression.getLeft()))) {
-              return new RowLevelFilterResolverImpl(expression, isExpressionResolve, false,
-                  tableIdentifier);
-            }
-            if (expressionTree.getFilterExpressionType() == ExpressionType.GREATERTHAN
-                || expressionTree.getFilterExpressionType() == ExpressionType.LESSTHAN
-                || expressionTree.getFilterExpressionType() == ExpressionType.GREATERTHAN_EQUALTO
-                || expressionTree.getFilterExpressionType() == ExpressionType.LESSTHAN_EQUALTO) {
-
-              return new RowLevelRangeFilterResolverImpl(expression, isExpressionResolve, false,
-                  tableIdentifier);
-            }
-
-            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false);
-          }
-          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false);
-        }
-        break;
-      default:
-        condExpression = (ConditionalExpression) expression;
-        if (condExpression.isSingleDimension()
-            && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.ARRAY
-            && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
-            != DataType.STRUCT) {
-          condExpression = (ConditionalExpression) expression;
-          if (condExpression.isSingleDimension()) {
-            if (!condExpression.getColumnList().get(0).getCarbonColumn()
-                .hasEncoding(Encoding.DICTIONARY)) {
-              if (FilterUtil.checkIfExpressionContainsColumn(expression)) {
-                return new RowLevelFilterResolverImpl(expression, isExpressionResolve, false,
-                    tableIdentifier);
-              } else if (expressionTree.getFilterExpressionType() == ExpressionType.UNKNOWN) {
-                return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
-              }
-
-              return new ConditionalFilterResolverImpl(expression, true, true);
-            }
-          }
-        } else {
-          return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
-        }
-    }
-    return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filters/FilterProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filters/FilterProcessor.java b/core/src/main/java/org/carbondata/query/filters/FilterProcessor.java
deleted file mode 100644
index b3a8e9e..0000000
--- a/core/src/main/java/org/carbondata/query/filters/FilterProcessor.java
+++ /dev/null
@@ -1,60 +0,0 @@
-package org.carbondata.query.filters;
-
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-import java.util.List;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.FilterResolverIntf;
-
-public interface FilterProcessor {
-
-  /**
-   * API will provide the resolved form of filters based on the filter
-   * expression tree which is been passed.
-   *
-   * @param expressionTree  , filter expression tree
-   * @param tableIdentifier ,contains carbon store informations.
-   * @return
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  FilterResolverIntf getFilterResolver(Expression expressionTree,
-      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException;
-
-  /**
-   * This API is exposed inorder to get the required block reference node
-   * based on the filter.The block list will be send to the executer tasks inorder
-   * to apply filters.
-   *
-   * @param filterResolver DataBlock list with resolved filters
-   * @return list of DataRefNode.
-   * @throws QueryExecutionException
-   */
-  List<DataRefNode> getFilterredBlocks(DataRefNode dataRefNode, FilterResolverIntf filterResolver,
-      AbstractIndex segmentIndexBuilder, AbsoluteTableIdentifier tableIdentifier)
-      throws QueryExecutionException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java b/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java
deleted file mode 100644
index 65ff97f..0000000
--- a/core/src/main/java/org/carbondata/query/filters/measurefilter/util/FilterUtil.java
+++ /dev/null
@@ -1,1234 +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.carbondata.query.filters.measurefilter.util;
-
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.Date;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.SortedMap;
-import java.util.TreeMap;
-import java.util.TreeSet;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.cache.Cache;
-import org.carbondata.core.cache.CacheProvider;
-import org.carbondata.core.cache.CacheType;
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.cache.dictionary.DictionaryChunksWrapper;
-import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.carbondata.core.cache.dictionary.ForwardDictionary;
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.IndexKey;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.core.util.CarbonUtilException;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.carbonfilterinterface.RowImpl;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.evaluators.DimColumnExecuterFilterInfo;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.LiteralExpression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.executer.AndFilterExecuterImpl;
-import org.carbondata.query.filter.executer.ColGroupFilterExecuterImpl;
-import org.carbondata.query.filter.executer.ExcludeFilterExecuterImpl;
-import org.carbondata.query.filter.executer.FilterExecuter;
-import org.carbondata.query.filter.executer.IncludeFilterExecuterImpl;
-import org.carbondata.query.filter.executer.OrFilterExecuterImpl;
-import org.carbondata.query.filter.executer.RestructureFilterExecuterImpl;
-import org.carbondata.query.filter.executer.RowLevelFilterExecuterImpl;
-import org.carbondata.query.filter.executer.RowLevelRangeTypeExecuterFacory;
-import org.carbondata.query.filter.resolver.FilterResolverIntf;
-import org.carbondata.query.filter.resolver.RowLevelFilterResolverImpl;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-public final class FilterUtil {
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(FilterUtil.class.getName());
-
-  private FilterUtil() {
-
-  }
-
-  /**
-   * Pattern used : Visitor Pattern
-   * Method will create filter executer tree based on the filter resolved tree,
-   * in this algorithm based on the resolver instance the executers will be visited
-   * and the resolved surrogates will be converted to keys
-   *
-   * @param filterExpressionResolverTree
-   * @param segmentProperties
-   * @return FilterExecuter instance
-   */
-  private static FilterExecuter createFilterExecuterTree(
-      FilterResolverIntf filterExpressionResolverTree, SegmentProperties segmentProperties) {
-    FilterExecuterType filterExecuterType = filterExpressionResolverTree.getFilterExecuterType();
-    switch (filterExecuterType) {
-      case INCLUDE:
-        return getIncludeFilterExecuter(filterExpressionResolverTree.getDimColResolvedFilterInfo(),
-            segmentProperties);
-      case EXCLUDE:
-        return new ExcludeFilterExecuterImpl(
-            filterExpressionResolverTree.getDimColResolvedFilterInfo(),
-            segmentProperties.getDimensionKeyGenerator());
-      case OR:
-        return new OrFilterExecuterImpl(
-            createFilterExecuterTree(filterExpressionResolverTree.getLeft(), segmentProperties),
-            createFilterExecuterTree(filterExpressionResolverTree.getRight(), segmentProperties));
-      case AND:
-        return new AndFilterExecuterImpl(
-            createFilterExecuterTree(filterExpressionResolverTree.getLeft(), segmentProperties),
-            createFilterExecuterTree(filterExpressionResolverTree.getRight(), segmentProperties));
-      case RESTRUCTURE:
-        return new RestructureFilterExecuterImpl(
-            filterExpressionResolverTree.getDimColResolvedFilterInfo(),
-            segmentProperties.getDimensionKeyGenerator());
-      case ROWLEVEL_LESSTHAN:
-      case ROWLEVEL_LESSTHAN_EQUALTO:
-      case ROWLEVEL_GREATERTHAN_EQUALTO:
-      case ROWLEVEL_GREATERTHAN:
-        return RowLevelRangeTypeExecuterFacory
-            .getRowLevelRangeTypeExecuter(filterExecuterType, filterExpressionResolverTree);
-      case ROWLEVEL:
-      default:
-        return new RowLevelFilterExecuterImpl(
-            ((RowLevelFilterResolverImpl) filterExpressionResolverTree)
-                .getDimColEvaluatorInfoList(),
-            ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getMsrColEvalutorInfoList(),
-            ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getFilterExpresion(),
-            ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier());
-
-    }
-
-  }
-
-  /**
-   * It gives filter executer based on columnar or column group
-   *
-   * @param dimColResolvedFilterInfo
-   * @param segmentProperties
-   * @return
-   */
-  private static FilterExecuter getIncludeFilterExecuter(
-      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties) {
-
-    if (dimColResolvedFilterInfo.getDimension().isColumnar()) {
-      return new IncludeFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
-    } else {
-      return new ColGroupFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
-    }
-  }
-
-  /**
-   * This method will check if a given expression contains a column expression
-   * recursively.
-   *
-   * @return
-   */
-  public static boolean checkIfExpressionContainsColumn(Expression expression) {
-    if (expression instanceof ColumnExpression) {
-      return true;
-    }
-    for (Expression child : expression.getChildren()) {
-      if (checkIfExpressionContainsColumn(child)) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  /**
-   * This method will check if a given expression contains a column expression
-   * recursively.
-   *
-   * @return
-   */
-  public static boolean checkIfExpressionContainsUnknownExp(Expression expression) {
-    if (expression.getFilterExpressionType() == ExpressionType.UNKNOWN) {
-      return true;
-    }
-    for (Expression child : expression.getChildren()) {
-      if (checkIfExpressionContainsUnknownExp(child)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * method will get the masked keys based on the keys generated from surrogates.
-   *
-   * @param ranges
-   * @param key
-   * @return byte[]
-   */
-  private static byte[] getMaskedKey(int[] ranges, byte[] key) {
-    byte[] maskkey = new byte[ranges.length];
-
-    for (int i = 0; i < maskkey.length; i++) {
-      maskkey[i] = key[ranges[i]];
-    }
-    return maskkey;
-  }
-
-  /**
-   * This method will return the ranges for the masked Bytes based on the key
-   * Generator.
-   *
-   * @param queryDimensionsOrdinal
-   * @param generator
-   * @return
-   */
-  private static int[] getRangesForMaskedByte(int queryDimensionsOrdinal, KeyGenerator generator) {
-    Set<Integer> integers = new TreeSet<Integer>();
-    int[] range = generator.getKeyByteOffsets(queryDimensionsOrdinal);
-    for (int j = range[0]; j <= range[1]; j++) {
-      integers.add(j);
-    }
-
-    int[] byteIndexs = new int[integers.size()];
-    int j = 0;
-    for (Iterator<Integer> iterator = integers.iterator(); iterator.hasNext(); ) {
-      Integer integer = iterator.next();
-      byteIndexs[j++] = integer.intValue();
-    }
-    return byteIndexs;
-  }
-
-  /**
-   * This method will get the no dictionary data based on filters and same
-   * will be in DimColumnFilterInfo
-   *
-   * @param tableIdentifier
-   * @param columnExpression
-   * @param evaluateResultListFinal
-   * @param isIncludeFilter
-   * @return DimColumnFilterInfo
-   */
-  public static DimColumnFilterInfo getNoDictionaryValKeyMemberForFilter(
-      AbsoluteTableIdentifier tableIdentifier, ColumnExpression columnExpression,
-      List<String> evaluateResultListFinal, boolean isIncludeFilter) {
-    List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
-    for (String result : evaluateResultListFinal) {
-      filterValuesList.add(result.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
-    }
-
-    Comparator<byte[]> filterNoDictValueComaparator = new Comparator<byte[]>() {
-
-      @Override public int compare(byte[] filterMember1, byte[] filterMember2) {
-        // TODO Auto-generated method stub
-        return ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterMember1, filterMember2);
-      }
-
-    };
-    Collections.sort(filterValuesList, filterNoDictValueComaparator);
-    DimColumnFilterInfo columnFilterInfo = null;
-    if (filterValuesList.size() > 0) {
-      columnFilterInfo = new DimColumnFilterInfo();
-      columnFilterInfo.setIncludeFilter(isIncludeFilter);
-      columnFilterInfo.setFilterListForNoDictionaryCols(filterValuesList);
-
-    }
-    return columnFilterInfo;
-  }
-
-  /**
-   * Method will prepare the  dimfilterinfo instance by resolving the filter
-   * expression value to its respective surrogates.
-   *
-   * @param tableIdentifier
-   * @param columnExpression
-   * @param evaluateResultList
-   * @param isIncludeFilter
-   * @return
-   * @throws QueryExecutionException
-   */
-  public static DimColumnFilterInfo getFilterValues(AbsoluteTableIdentifier tableIdentifier,
-      ColumnExpression columnExpression, List<String> evaluateResultList, boolean isIncludeFilter)
-      throws QueryExecutionException, FilterUnsupportedException {
-    Dictionary forwardDictionary = null;
-    try {
-      // Reading the dictionary value from cache.
-      forwardDictionary =
-          getForwardDictionaryCache(tableIdentifier, columnExpression.getDimension());
-      return getFilterValues(columnExpression, evaluateResultList, forwardDictionary,
-          isIncludeFilter);
-    } finally {
-      CarbonUtil.clearDictionaryCache(forwardDictionary);
-    }
-  }
-
-  /**
-   * Method will prepare the  dimfilterinfo instance by resolving the filter
-   * expression value to its respective surrogates.
-   *
-   * @param columnExpression
-   * @param evaluateResultList
-   * @param forwardDictionary
-   * @param isIncludeFilter
-   * @return
-   * @throws QueryExecutionException
-   */
-  private static DimColumnFilterInfo getFilterValues(ColumnExpression columnExpression,
-      List<String> evaluateResultList, Dictionary forwardDictionary, boolean isIncludeFilter)
-      throws QueryExecutionException {
-    sortFilterModelMembers(columnExpression, evaluateResultList);
-    List<Integer> surrogates =
-        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // Reading the dictionary value from cache.
-    getDictionaryValue(evaluateResultList, forwardDictionary, surrogates);
-    Collections.sort(surrogates);
-    DimColumnFilterInfo columnFilterInfo = null;
-    if (surrogates.size() > 0) {
-      columnFilterInfo = new DimColumnFilterInfo();
-      columnFilterInfo.setIncludeFilter(isIncludeFilter);
-      columnFilterInfo.setFilterList(surrogates);
-    }
-    return columnFilterInfo;
-  }
-
-  /**
-   * This API will get the Dictionary value for the respective filter member
-   * string.
-   *
-   * @param evaluateResultList filter value
-   * @param surrogates
-   * @throws QueryExecutionException
-   */
-  private static void getDictionaryValue(List<String> evaluateResultList,
-      Dictionary forwardDictionary, List<Integer> surrogates) throws QueryExecutionException {
-    ((ForwardDictionary) forwardDictionary)
-        .getSurrogateKeyByIncrementalSearch(evaluateResultList, surrogates);
-  }
-
-  /**
-   * This method will get all the members of column from the forward dictionary
-   * cache, this method will be basically used in row level filter resolver.
-   *
-   * @param tableIdentifier
-   * @param expression
-   * @param columnExpression
-   * @param isIncludeFilter
-   * @return DimColumnFilterInfo
-   * @throws QueryExecutionException
-   */
-  public static DimColumnFilterInfo getFilterListForAllValues(
-      AbsoluteTableIdentifier tableIdentifier, Expression expression,
-      final ColumnExpression columnExpression, boolean isIncludeFilter)
-      throws FilterUnsupportedException {
-    Dictionary forwardDictionary = null;
-    List<String> evaluateResultListFinal = new ArrayList<String>(20);
-    DictionaryChunksWrapper dictionaryWrapper = null;
-    try {
-      forwardDictionary =
-          getForwardDictionaryCache(tableIdentifier, columnExpression.getDimension());
-      dictionaryWrapper = forwardDictionary.getDictionaryChunks();
-      while (dictionaryWrapper.hasNext()) {
-        byte[] columnVal = dictionaryWrapper.next();
-        try {
-          RowIntf row = new RowImpl();
-          String stringValue =
-              new String(columnVal, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-          if (stringValue.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
-            stringValue = null;
-          }
-          row.setValues(new Object[] { DataTypeUtil.getDataBasedOnDataType(stringValue,
-              columnExpression.getCarbonColumn().getDataType()) });
-          Boolean rslt = expression.evaluate(row).getBoolean();
-          if (null != rslt && !(rslt ^ isIncludeFilter)) {
-            if (null == stringValue) {
-              evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-            } else {
-              evaluateResultListFinal.add(stringValue);
-            }
-          }
-        } catch (FilterUnsupportedException e) {
-          LOGGER.audit(e.getMessage());
-          throw new FilterUnsupportedException(e.getMessage());
-        }
-      }
-      return getFilterValues(columnExpression, evaluateResultListFinal, forwardDictionary,
-          isIncludeFilter);
-    } catch (QueryExecutionException e) {
-      throw new FilterUnsupportedException(e.getMessage());
-    } finally {
-      CarbonUtil.clearDictionaryCache(forwardDictionary);
-    }
-  }
-
-  private static void sortFilterModelMembers(final ColumnExpression columnExpression,
-      List<String> evaluateResultListFinal) {
-    Comparator<String> filterActualValueComaparator = new Comparator<String>() {
-
-      @Override public int compare(String filterMember1, String filterMember2) {
-        return compareFilterMembersBasedOnActualDataType(filterMember1, filterMember2,
-            columnExpression.getDataType());
-      }
-
-    };
-    Collections.sort(evaluateResultListFinal, filterActualValueComaparator);
-  }
-
-  /**
-   * Metahod will resolve the filter member to its respective surrogates by
-   * scanning the dictionary cache.
-   *
-   * @param tableIdentifier
-   * @param expression
-   * @param columnExpression
-   * @param isIncludeFilter
-   * @return
-   * @throws QueryExecutionException
-   */
-  public static DimColumnFilterInfo getFilterList(AbsoluteTableIdentifier tableIdentifier,
-      Expression expression, ColumnExpression columnExpression, boolean isIncludeFilter)
-      throws QueryExecutionException {
-    DimColumnFilterInfo resolvedFilterObject = null;
-    List<String> evaluateResultListFinal = new ArrayList<String>(20);
-    try {
-      List<ExpressionResult> evaluateResultList = expression.evaluate(null).getList();
-      for (ExpressionResult result : evaluateResultList) {
-        if (result.getString() == null) {
-          evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-          continue;
-        }
-        evaluateResultListFinal.add(result.getString());
-      }
-
-      if (null != columnExpression.getCarbonColumn() && !columnExpression.getCarbonColumn()
-          .hasEncoding(Encoding.DICTIONARY)) {
-        resolvedFilterObject =
-            getNoDictionaryValKeyMemberForFilter(tableIdentifier, columnExpression,
-                evaluateResultListFinal, isIncludeFilter);
-      } else {
-        resolvedFilterObject =
-            getFilterValues(tableIdentifier, columnExpression, evaluateResultListFinal,
-                isIncludeFilter);
-      }
-    } catch (FilterUnsupportedException e) {
-      LOGGER.audit(e.getMessage());
-    }
-    return resolvedFilterObject;
-  }
-
-  /**
-   * Method will prepare the  dimfilterinfo instance by resolving the filter
-   * expression value to its respective surrogates in the scenario of restructure.
-   *
-   * @param expression
-   * @param columnExpression
-   * @param defaultValues
-   * @param defaultSurrogate
-   * @return
-   */
-  public static DimColumnFilterInfo getFilterListForRS(Expression expression,
-      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate) {
-    List<Integer> filterValuesList = new ArrayList<Integer>(20);
-    DimColumnFilterInfo columnFilterInfo = null;
-    // List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
-    List<String> evaluateResultListFinal = new ArrayList<String>(20);
-    // KeyGenerator keyGenerator =
-    // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate });
-    try {
-      List<ExpressionResult> evaluateResultList = expression.evaluate(null).getList();
-      for (ExpressionResult result : evaluateResultList) {
-        if (result.getString() == null) {
-          evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-          continue;
-        }
-        evaluateResultListFinal.add(result.getString());
-      }
-
-      for (int i = 0; i < evaluateResultListFinal.size(); i++) {
-        if (evaluateResultListFinal.get(i).equals(defaultValues)) {
-          filterValuesList.add(defaultSurrogate);
-          break;
-        }
-      }
-      if (filterValuesList.size() > 0) {
-        columnFilterInfo = new DimColumnFilterInfo();
-        columnFilterInfo.setFilterList(filterValuesList);
-      }
-    } catch (FilterUnsupportedException e) {
-      LOGGER.audit(e.getMessage());
-    }
-    return columnFilterInfo;
-  }
-
-  /**
-   * This method will get the member based on filter expression evaluation from the
-   * forward dictionary cache, this method will be basically used in restructure.
-   *
-   * @param expression
-   * @param columnExpression
-   * @param defaultValues
-   * @param defaultSurrogate
-   * @param isIncludeFilter
-   * @return
-   */
-  public static DimColumnFilterInfo getFilterListForAllMembersRS(Expression expression,
-      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate,
-      boolean isIncludeFilter) {
-    List<Integer> filterValuesList = new ArrayList<Integer>(20);
-    List<String> evaluateResultListFinal = new ArrayList<String>(20);
-    DimColumnFilterInfo columnFilterInfo = null;
-
-    // KeyGenerator keyGenerator =
-    // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate });
-    try {
-      RowIntf row = new RowImpl();
-      if (defaultValues.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
-        defaultValues = null;
-      }
-      row.setValues(new Object[] { DataTypeUtil.getDataBasedOnDataType(defaultValues,
-          columnExpression.getCarbonColumn().getDataType()) });
-      Boolean rslt = expression.evaluate(row).getBoolean();
-      if (null != rslt && !(rslt ^ isIncludeFilter)) {
-        if (null == defaultValues) {
-          evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-        } else {
-          evaluateResultListFinal.add(defaultValues);
-        }
-      }
-    } catch (FilterUnsupportedException e) {
-      LOGGER.audit(e.getMessage());
-    }
-
-    if (null == defaultValues) {
-      defaultValues = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
-    }
-    columnFilterInfo = new DimColumnFilterInfo();
-    for (int i = 0; i < evaluateResultListFinal.size(); i++) {
-      if (evaluateResultListFinal.get(i).equals(defaultValues)) {
-        filterValuesList.add(defaultSurrogate);
-        break;
-      }
-    }
-    columnFilterInfo.setFilterList(filterValuesList);
-    return columnFilterInfo;
-  }
-
-  public static byte[][] getKeyArray(DimColumnFilterInfo dimColumnFilterInfo,
-      CarbonDimension carbonDimension, KeyGenerator blockLevelKeyGenerator) {
-    if (!carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
-      return dimColumnFilterInfo.getNoDictionaryFilterValuesList()
-          .toArray((new byte[dimColumnFilterInfo.getNoDictionaryFilterValuesList().size()][]));
-    }
-    int[] keys = new int[blockLevelKeyGenerator.getDimCount()];
-    List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
-    Arrays.fill(keys, 0);
-    int[] rangesForMaskedByte =
-        getRangesForMaskedByte((carbonDimension.getKeyOrdinal()), blockLevelKeyGenerator);
-    if (null != dimColumnFilterInfo) {
-      for (Integer surrogate : dimColumnFilterInfo.getFilterList()) {
-        try {
-          keys[carbonDimension.getKeyOrdinal()] = surrogate;
-          filterValuesList
-              .add(getMaskedKey(rangesForMaskedByte, blockLevelKeyGenerator.generateKey(keys)));
-        } catch (KeyGenException e) {
-          LOGGER.error(e.getMessage());
-        }
-      }
-    }
-    return filterValuesList.toArray(new byte[filterValuesList.size()][]);
-
-  }
-
-  /**
-   * Method will return the start key based on KeyGenerator for the respective
-   * filter resolved instance.
-   *
-   * @param dimColResolvedFilterInfo
-   * @param segmentProperties
-   * @return long[] start key
-   */
-  public static long[] getStartKey(DimColumnResolvedFilterInfo dimColResolvedFilterInfo,
-      SegmentProperties segmentProperties, long[] startKey) {
-    Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter =
-        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
-    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      List<DimColumnFilterInfo> values = entry.getValue();
-      if (null == values || !entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
-        continue;
-      }
-      boolean isExcludePresent = false;
-      for (DimColumnFilterInfo info : values) {
-        if (!info.isIncludeFilter()) {
-          isExcludePresent = true;
-        }
-      }
-      if (isExcludePresent) {
-        continue;
-      }
-      getStartKeyBasedOnFilterResoverInfo(dimensionFilter, startKey);
-    }
-    return startKey;
-  }
-
-  /**
-   * Algorithm for getting the start key for a filter
-   * step 1: Iterate through each dimension and verify whether its not an exclude filter.
-   * step 2: Intialize start key with the first filter member value present in each filter model
-   * for the respective dimensions.
-   * step 3: since its a no dictionary start key there will only actual value so compare
-   * the first filter model value with respect to the dimension data type.
-   * step 4: The least value will be considered as the start key of dimension by comparing all
-   * its filter model.
-   * step 5: create a byte array of start key which comprises of least filter member value of
-   * all dimension and the indexes which will help to read the respective filter value.
-   *
-   * @param dimColResolvedFilterInfo
-   * @param segmentProperties
-   * @param setOfStartKeyByteArray
-   * @return
-   */
-  public static void getStartKeyForNoDictionaryDimension(
-      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
-    Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter =
-        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
-    // step 1
-    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      if (!entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
-        List<DimColumnFilterInfo> listOfDimColFilterInfo = entry.getValue();
-        if (null == listOfDimColFilterInfo) {
-          continue;
-        }
-        boolean isExcludePresent = false;
-        for (DimColumnFilterInfo info : listOfDimColFilterInfo) {
-          if (!info.isIncludeFilter()) {
-            isExcludePresent = true;
-          }
-        }
-        if (isExcludePresent) {
-          continue;
-        }
-        // step 2
-        byte[] noDictionaryStartKey =
-            listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList().get(0);
-        if (setOfStartKeyByteArray.isEmpty()) {
-          setOfStartKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryStartKey);
-        } else if (null == setOfStartKeyByteArray.get(entry.getKey().getOrdinal())) {
-          setOfStartKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryStartKey);
-
-        } else if (ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(setOfStartKeyByteArray.get(entry.getKey().getOrdinal()),
-                noDictionaryStartKey) > 0) {
-          setOfStartKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryStartKey);
-        }
-      }
-    }
-  }
-
-  /**
-   * Algorithm for getting the end key for a filter
-   * step 1: Iterate through each dimension and verify whether its not an exclude filter.
-   * step 2: Initialize end key with the last filter member value present in each filter model
-   * for the respective dimensions.(Already filter models are sorted)
-   * step 3: since its a no dictionary end key there will only actual value so compare
-   * the last filter model value with respect to the dimension data type.
-   * step 4: The highest value will be considered as the end key of dimension by comparing all
-   * its filter model.
-   * step 5: create a byte array of end key which comprises of highest filter member value of
-   * all dimension and the indexes which will help to read the respective filter value.
-   *
-   * @param dimColResolvedFilterInfo
-   * @param segmentProperties
-   * @param setOfEndKeyByteArray
-   * @return end key array
-   */
-  public static void getEndKeyForNoDictionaryDimension(
-      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
-
-    Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter =
-        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
-    // step 1
-    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      if (!entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
-        List<DimColumnFilterInfo> listOfDimColFilterInfo = entry.getValue();
-        if (null == listOfDimColFilterInfo) {
-          continue;
-        }
-        boolean isExcludePresent = false;
-        for (DimColumnFilterInfo info : listOfDimColFilterInfo) {
-          if (!info.isIncludeFilter()) {
-            isExcludePresent = true;
-          }
-        }
-        if (isExcludePresent) {
-          continue;
-        }
-        // step 2
-        byte[] noDictionaryEndKey = listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList()
-            .get(listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList().size() - 1);
-        if (setOfEndKeyByteArray.isEmpty()) {
-          setOfEndKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryEndKey);
-        } else if (null == setOfEndKeyByteArray.get(entry.getKey().getOrdinal())) {
-          setOfEndKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryEndKey);
-
-        } else if (ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(setOfEndKeyByteArray.get(entry.getKey().getOrdinal()), noDictionaryEndKey)
-            < 0) {
-          setOfEndKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryEndKey);
-        }
-
-      }
-    }
-  }
-
-  /**
-   * Method will pack all the byte[] to a single byte[] value by appending the
-   * indexes of the byte[] value which needs to be read. this method will be mailny used
-   * in case of no dictionary dimension processing for filters.
-   *
-   * @param noDictionaryValKeyList
-   * @return packed key with its indexes added in starting and its actual values.
-   */
-  private static byte[] getKeyWithIndexesAndValues(List<byte[]> noDictionaryValKeyList) {
-    ByteBuffer[] buffArr = new ByteBuffer[noDictionaryValKeyList.size()];
-    int index = 0;
-    for (byte[] singleColVal : noDictionaryValKeyList) {
-      buffArr[index] = ByteBuffer.allocate(singleColVal.length);
-      buffArr[index].put(singleColVal);
-      buffArr[index++].rewind();
-    }
-    // byteBufer.
-    return CarbonUtil.packByteBufferIntoSingleByteArray(buffArr);
-
-  }
-
-  /**
-   * This method will fill the start key array  with the surrogate key present
-   * in filterinfo instance.
-   *
-   * @param dimensionFilter
-   * @param startKey
-   */
-  private static void getStartKeyBasedOnFilterResoverInfo(
-      Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter, long[] startKey) {
-    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      List<DimColumnFilterInfo> values = entry.getValue();
-      if (null == values) {
-        continue;
-      }
-      boolean isExcludePresent = false;
-      for (DimColumnFilterInfo info : values) {
-        if (!info.isIncludeFilter()) {
-          isExcludePresent = true;
-        }
-      }
-      if (isExcludePresent) {
-        continue;
-      }
-      for (DimColumnFilterInfo info : values) {
-        if (startKey[entry.getKey().getKeyOrdinal()] < info.getFilterList().get(0)) {
-          startKey[entry.getKey().getKeyOrdinal()] = info.getFilterList().get(0);
-        }
-      }
-    }
-  }
-
-  public static void getEndKey(Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter,
-      AbsoluteTableIdentifier tableIdentifier, long[] endKey, SegmentProperties segmentProperties)
-      throws QueryExecutionException {
-
-    List<CarbonDimension> updatedDimListBasedOnKeyGenerator =
-        getCarbonDimsMappedToKeyGenerator(segmentProperties.getDimensions());
-    for (int i = 0; i < endKey.length; i++) {
-      endKey[i] = getMaxValue(tableIdentifier, updatedDimListBasedOnKeyGenerator.get(i),
-          segmentProperties.getDimColumnsCardinality());
-    }
-    getEndKeyWithFilter(dimensionFilter, endKey);
-
-  }
-
-  private static List<CarbonDimension> getCarbonDimsMappedToKeyGenerator(
-      List<CarbonDimension> carbonDimensions) {
-    List<CarbonDimension> listOfCarbonDimPartOfKeyGen =
-        new ArrayList<CarbonDimension>(carbonDimensions.size());
-    for (CarbonDimension carbonDim : carbonDimensions) {
-      if (CarbonUtil.hasEncoding(carbonDim.getEncoder(), Encoding.DICTIONARY) || CarbonUtil
-          .hasEncoding(carbonDim.getEncoder(), Encoding.DIRECT_DICTIONARY)) {
-        listOfCarbonDimPartOfKeyGen.add(carbonDim);
-      }
-
-    }
-    return listOfCarbonDimPartOfKeyGen;
-  }
-
-  private static void getEndKeyWithFilter(
-      Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter, long[] endKey) {
-    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
-      List<DimColumnFilterInfo> values = entry.getValue();
-      if (null == values || !entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
-        continue;
-      }
-      boolean isExcludeFilterPresent = false;
-      for (DimColumnFilterInfo info : values) {
-        if (!info.isIncludeFilter()) {
-          isExcludeFilterPresent = true;
-        }
-      }
-      if (isExcludeFilterPresent) {
-        continue;
-      }
-
-      for (DimColumnFilterInfo info : values) {
-        if (endKey[entry.getKey().getKeyOrdinal()] > info.getFilterList()
-            .get(info.getFilterList().size() - 1)) {
-          endKey[entry.getKey().getKeyOrdinal()] =
-              info.getFilterList().get(info.getFilterList().size() - 1);
-        }
-      }
-    }
-
-  }
-
-  /**
-   * This API will get the max value of surrogate key which will be used for
-   * determining the end key of particular btree.
-   *
-   * @param dimCarinality
-   * @throws QueryExecutionException
-   */
-  private static long getMaxValue(AbsoluteTableIdentifier tableIdentifier,
-      CarbonDimension carbonDimension, int[] dimCarinality) throws QueryExecutionException {
-    //    if (DataType.TIMESTAMP == carbonDimension.getDataType()) {
-    //      return Integer.MAX_VALUE;
-    //    }
-    // Get data from all the available slices of the cube
-    if (null != dimCarinality) {
-      return dimCarinality[carbonDimension.getKeyOrdinal()];
-    }
-    return -1;
-  }
-
-  /**
-   * @param tableIdentifier
-   * @param carbonDimension
-   * @return
-   * @throws QueryExecutionException
-   */
-  public static Dictionary getForwardDictionaryCache(AbsoluteTableIdentifier tableIdentifier,
-      CarbonDimension carbonDimension) throws QueryExecutionException {
-    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-        new DictionaryColumnUniqueIdentifier(tableIdentifier.getCarbonTableIdentifier(),
-            String.valueOf(carbonDimension.getColumnId()), carbonDimension.getDataType());
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    Cache forwardDictionaryCache =
-        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, tableIdentifier.getStorePath());
-    // get the forward dictionary object
-    Dictionary forwardDictionary = null;
-    try {
-      forwardDictionary = (Dictionary) forwardDictionaryCache.get(dictionaryColumnUniqueIdentifier);
-    } catch (CarbonUtilException e) {
-      throw new QueryExecutionException(e);
-    }
-    return forwardDictionary;
-  }
-
-  public static IndexKey createIndexKeyFromResolvedFilterVal(long[] startOrEndKey,
-      KeyGenerator keyGenerator, byte[] startOrEndKeyForNoDictDimension) {
-    IndexKey indexKey = null;
-    try {
-      indexKey =
-          new IndexKey(keyGenerator.generateKey(startOrEndKey), startOrEndKeyForNoDictDimension);
-    } catch (KeyGenException e) {
-      LOGGER.error(e.getMessage());
-    }
-    return indexKey;
-  }
-
-  /**
-   * API will create an filter executer tree based on the filter resolver
-   *
-   * @param filterExpressionResolverTree
-   * @param segmentProperties
-   * @return
-   */
-  public static FilterExecuter getFilterExecuterTree(
-      FilterResolverIntf filterExpressionResolverTree, SegmentProperties segmentProperties) {
-    return createFilterExecuterTree(filterExpressionResolverTree, segmentProperties);
-  }
-
-  /**
-   * API will prepare the Keys from the surrogates of particular filter resolver
-   *
-   * @param filterValues
-   * @param blockKeyGenerator
-   * @param dimension
-   * @param dimColumnExecuterInfo
-   */
-  public static void prepareKeysFromSurrogates(DimColumnFilterInfo filterValues,
-      KeyGenerator blockKeyGenerator, CarbonDimension dimension,
-      DimColumnExecuterFilterInfo dimColumnExecuterInfo) {
-    byte[][] keysBasedOnFilter = getKeyArray(filterValues, dimension, blockKeyGenerator);
-    dimColumnExecuterInfo.setFilterKeys(keysBasedOnFilter);
-
-  }
-
-  /**
-   * method will create a default end key in case of no end key is been derived using existing
-   * filter or in case of non filter queries.
-   *
-   * @param segmentProperties
-   * @return
-   * @throws KeyGenException
-   */
-  public static IndexKey prepareDefaultEndIndexKey(SegmentProperties segmentProperties)
-      throws KeyGenException {
-    long[] dictionarySurrogateKey =
-        new long[segmentProperties.getDimensions().size() - segmentProperties
-            .getNumberOfNoDictionaryDimension()];
-    Arrays.fill(dictionarySurrogateKey, Long.MAX_VALUE);
-    IndexKey endIndexKey;
-    byte[] dictionaryendMdkey =
-        segmentProperties.getDimensionKeyGenerator().generateKey(dictionarySurrogateKey);
-    byte[] noDictionaryEndKeyBuffer = getNoDictionaryDefaultEndKey(segmentProperties);
-    endIndexKey = new IndexKey(dictionaryendMdkey, noDictionaryEndKeyBuffer);
-    return endIndexKey;
-  }
-
-  public static byte[] getNoDictionaryDefaultEndKey(SegmentProperties segmentProperties) {
-    // in case of non filter query when no dictionary columns are present we
-    // need to set the default end key, as for non filter query
-    // we need to get the last
-    // block of the btree so we are setting the max byte value in the end key
-    ByteBuffer noDictionaryEndKeyBuffer = ByteBuffer.allocate(
-        (segmentProperties.getNumberOfNoDictionaryDimension()
-            * CarbonCommonConstants.SHORT_SIZE_IN_BYTE) + segmentProperties
-            .getNumberOfNoDictionaryDimension());
-    // end key structure will be
-    //<Offset of first No Dictionary key in 2 Bytes><Offset of second No Dictionary key in 2 Bytes>
-    //<Offset of n No Dictionary key in 2 Bytes><first no dictionary column value>
-    // <second no dictionary column value> <N no dictionary column value>
-    //example if we have 2 no dictionary column
-    //<[0,4,0,5,127,127]>
-    short startPoint = (short) (segmentProperties.getNumberOfNoDictionaryDimension()
-        * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
-    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
-      noDictionaryEndKeyBuffer.putShort((startPoint));
-      startPoint++;
-    }
-    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
-      noDictionaryEndKeyBuffer.put((byte) 127);
-    }
-    return noDictionaryEndKeyBuffer.array();
-  }
-
-  /**
-   * method will create a default end key in case of no end key is been
-   * derived using existing filter or in case of non filter queries.
-   *
-   * @param segmentProperties
-   * @return
-   * @throws KeyGenException
-   */
-  public static IndexKey prepareDefaultStartIndexKey(SegmentProperties segmentProperties)
-      throws KeyGenException {
-    IndexKey startIndexKey;
-    long[] dictionarySurrogateKey =
-        new long[segmentProperties.getDimensions().size() - segmentProperties
-            .getNumberOfNoDictionaryDimension()];
-    byte[] dictionaryStartMdkey =
-        segmentProperties.getDimensionKeyGenerator().generateKey(dictionarySurrogateKey);
-    byte[] noDictionaryStartKeyArray = getNoDictionaryDefaultStartKey(segmentProperties);
-
-    startIndexKey = new IndexKey(dictionaryStartMdkey, noDictionaryStartKeyArray);
-    return startIndexKey;
-  }
-
-  public static byte[] getNoDictionaryDefaultStartKey(SegmentProperties segmentProperties) {
-    // in case of non filter query when no dictionary columns are present we
-    // need to set the default start key, as for non filter query we need to get the first
-    // block of the btree so we are setting the least byte value in the start key
-    ByteBuffer noDictionaryStartKeyBuffer = ByteBuffer.allocate(
-        (segmentProperties.getNumberOfNoDictionaryDimension()
-            * CarbonCommonConstants.SHORT_SIZE_IN_BYTE) + segmentProperties
-            .getNumberOfNoDictionaryDimension());
-    // end key structure will be
-    //<Offset of first No Dictionary key in 2 Bytes><Offset of second No Dictionary key in 2 Bytes>
-    //<Offset of n No Dictionary key in 2 Bytes><first no dictionary column value>
-    // <second no dictionary column value> <N no dictionary column value>
-    //example if we have 2 no dictionary column
-    //<[0,4,0,5,0,0]>
-    short startPoint = (short) (segmentProperties.getNumberOfNoDictionaryDimension()
-        * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
-    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
-      noDictionaryStartKeyBuffer.putShort((startPoint));
-      startPoint++;
-    }
-    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
-      noDictionaryStartKeyBuffer.put((byte) 0);
-    }
-    return noDictionaryStartKeyBuffer.array();
-  }
-
-  public static int compareFilterKeyBasedOnDataType(String dictionaryVal, String memberVal,
-      DataType dataType) {
-    try {
-      switch (dataType) {
-        case INT:
-
-          return Integer.compare((Integer.parseInt(dictionaryVal)), (Integer.parseInt(memberVal)));
-        case DOUBLE:
-          return Double
-              .compare((Double.parseDouble(dictionaryVal)), (Double.parseDouble(memberVal)));
-        case LONG:
-          return Long.compare((Long.parseLong(dictionaryVal)), (Long.parseLong(memberVal)));
-        case BOOLEAN:
-          return Boolean
-              .compare((Boolean.parseBoolean(dictionaryVal)), (Boolean.parseBoolean(memberVal)));
-        case TIMESTAMP:
-          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-          Date dateToStr;
-          Date dictionaryDate;
-          dateToStr = parser.parse(memberVal);
-          dictionaryDate = parser.parse(dictionaryVal);
-          return dictionaryDate.compareTo(dateToStr);
-
-        case DECIMAL:
-          java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
-          java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
-          return javaDecValForDictVal.compareTo(javaDecValForMemberVal);
-        default:
-          return -1;
-      }
-    } catch (Exception e) {
-      return -1;
-    }
-  }
-
-  /**
-   * method will set the start and end key for as per the filter resolver tree
-   * utilized visitor pattern inorder to populate the start and end key population.
-   *
-   * @param segmentProperties
-   * @param tableIdentifier
-   * @param filterResolver
-   * @param listOfStartEndKeys
-   * @throws QueryExecutionException
-   */
-  public static void traverseResolverTreeAndGetStartAndEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier tableIdentifier, FilterResolverIntf filterResolver,
-      List<IndexKey> listOfStartEndKeys) throws QueryExecutionException {
-    IndexKey searchStartKey = null;
-    IndexKey searchEndKey = null;
-    long[] startKey = new long[segmentProperties.getDimensionKeyGenerator().getDimCount()];
-    long[] endKey = new long[segmentProperties.getDimensionKeyGenerator().getDimCount()];
-    List<byte[]> listOfStartKeyByteArray =
-        new ArrayList<byte[]>(segmentProperties.getNumberOfNoDictionaryDimension());
-    List<byte[]> listOfEndKeyByteArray =
-        new ArrayList<byte[]>(segmentProperties.getNumberOfNoDictionaryDimension());
-    SortedMap<Integer, byte[]> setOfStartKeyByteArray = new TreeMap<Integer, byte[]>();
-    SortedMap<Integer, byte[]> setOfEndKeyByteArray = new TreeMap<Integer, byte[]>();
-    SortedMap<Integer, byte[]> defaultStartValues = new TreeMap<Integer, byte[]>();
-    SortedMap<Integer, byte[]> defaultEndValues = new TreeMap<Integer, byte[]>();
-    traverseResolverTreeAndPopulateStartAndEndKeys(filterResolver, tableIdentifier,
-        segmentProperties, startKey, setOfStartKeyByteArray, endKey, setOfEndKeyByteArray);
-    fillDefaultStartValue(defaultStartValues, segmentProperties);
-    fillDefaultEndValue(defaultEndValues, segmentProperties);
-    fillNullValuesStartIndexWithDefaultKeys(setOfStartKeyByteArray, segmentProperties);
-    fillNullValuesEndIndexWithDefaultKeys(setOfEndKeyByteArray, segmentProperties);
-    pruneStartAndEndKeys(setOfStartKeyByteArray, listOfStartKeyByteArray);
-    pruneStartAndEndKeys(setOfEndKeyByteArray, listOfEndKeyByteArray);
-
-    searchStartKey = FilterUtil
-        .createIndexKeyFromResolvedFilterVal(startKey, segmentProperties.getDimensionKeyGenerator(),
-            FilterUtil.getKeyWithIndexesAndValues(listOfStartKeyByteArray));
-
-    searchEndKey = FilterUtil
-        .createIndexKeyFromResolvedFilterVal(endKey, segmentProperties.getDimensionKeyGenerator(),
-            FilterUtil.getKeyWithIndexesAndValues(listOfEndKeyByteArray));
-    listOfStartEndKeys.add(searchStartKey);
-    listOfStartEndKeys.add(searchEndKey);
-
-  }
-
-  private static int compareFilterMembersBasedOnActualDataType(String filterMember1,
-      String filterMember2, org.carbondata.query.expression.DataType dataType) {
-    try {
-      switch (dataType) {
-        case IntegerType:
-        case LongType:
-        case DoubleType:
-
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
-            return 1;
-          }
-          Double d1 = Double.parseDouble(filterMember1);
-          Double d2 = Double.parseDouble(filterMember2);
-          return d1.compareTo(d2);
-        case DecimalType:
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
-            return 1;
-          }
-          java.math.BigDecimal val1 = new BigDecimal(filterMember1);
-          java.math.BigDecimal val2 = new BigDecimal(filterMember2);
-          return val1.compareTo(val2);
-        case TimestampType:
-          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
-            return 1;
-          }
-          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-          Date date1 = null;
-          Date date2 = null;
-          date1 = parser.parse(filterMember1);
-          date2 = parser.parse(filterMember2);
-          return date1.compareTo(date2);
-        case StringType:
-        default:
-          return filterMember1.compareTo(filterMember2);
-      }
-    } catch (Exception e) {
-      return -1;
-    }
-  }
-
-  private static void fillNullValuesStartIndexWithDefaultKeys(
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray, SegmentProperties segmentProperties) {
-    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
-    for (CarbonDimension dimension : allDimension) {
-      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
-        continue;
-      }
-      if (null == setOfStartKeyByteArray.get(dimension.getOrdinal())) {
-        setOfStartKeyByteArray.put(dimension.getOrdinal(), new byte[] { 0 });
-      }
-
-    }
-  }
-
-  private static void fillNullValuesEndIndexWithDefaultKeys(
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray, SegmentProperties segmentProperties) {
-    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
-    for (CarbonDimension dimension : allDimension) {
-      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
-        continue;
-      }
-      if (null == setOfStartKeyByteArray.get(dimension.getOrdinal())) {
-        setOfStartKeyByteArray.put(dimension.getOrdinal(), new byte[] { 127 });
-      }
-
-    }
-  }
-
-  private static void pruneStartAndEndKeys(SortedMap<Integer, byte[]> setOfStartKeyByteArray,
-      List<byte[]> listOfStartKeyByteArray) {
-    for (Map.Entry<Integer, byte[]> entry : setOfStartKeyByteArray.entrySet()) {
-      listOfStartKeyByteArray.add(entry.getValue());
-    }
-  }
-
-  private static void fillDefaultStartValue(SortedMap<Integer, byte[]> setOfStartKeyByteArray,
-      SegmentProperties segmentProperties) {
-    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
-    for (CarbonDimension dimension : allDimension) {
-      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
-        continue;
-      }
-      setOfStartKeyByteArray.put(dimension.getOrdinal(), new byte[] { 0 });
-    }
-
-  }
-
-  private static void fillDefaultEndValue(SortedMap<Integer, byte[]> setOfEndKeyByteArray,
-      SegmentProperties segmentProperties) {
-    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
-    for (CarbonDimension dimension : allDimension) {
-      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
-        continue;
-      }
-      setOfEndKeyByteArray.put(dimension.getOrdinal(), new byte[] { 127 });
-    }
-  }
-
-  private static void traverseResolverTreeAndPopulateStartAndEndKeys(
-      FilterResolverIntf filterResolverTree, AbsoluteTableIdentifier tableIdentifier,
-      SegmentProperties segmentProperties, long[] startKeys,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) throws QueryExecutionException {
-    if (null == filterResolverTree) {
-      return;
-    }
-    traverseResolverTreeAndPopulateStartAndEndKeys(filterResolverTree.getLeft(), tableIdentifier,
-        segmentProperties, startKeys, setOfStartKeyByteArray, endKeys, setOfEndKeyByteArray);
-
-    filterResolverTree.getStartKey(segmentProperties, startKeys, setOfStartKeyByteArray);
-    filterResolverTree.getEndKey(segmentProperties, tableIdentifier, endKeys, setOfEndKeyByteArray);
-
-    traverseResolverTreeAndPopulateStartAndEndKeys(filterResolverTree.getRight(), tableIdentifier,
-        segmentProperties, startKeys, setOfStartKeyByteArray, endKeys, setOfEndKeyByteArray);
-  }
-
-  /**
-   * Method will find whether the expression needs to be resolved, this can happen
-   * if the expression is exclude and data type is null(mainly in IS NOT NULL filter scenario)
-   * @param rightExp
-   * @param isIncludeFilter
-   * @return
-   */
-  public static boolean isExpressionNeedsToResolved(Expression rightExp, boolean isIncludeFilter) {
-    if (!isIncludeFilter && rightExp instanceof LiteralExpression && (
-        org.carbondata.query.expression.DataType.NullType == ((LiteralExpression) rightExp)
-            .getLiteralExpDataType())) {
-      return true;
-    }
-    for (Expression child : rightExp.getChildren()) {
-      if (isExpressionNeedsToResolved(child, isIncludeFilter)) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/schema/metadata/DimColumnFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/schema/metadata/DimColumnFilterInfo.java b/core/src/main/java/org/carbondata/query/schema/metadata/DimColumnFilterInfo.java
deleted file mode 100644
index c25fe58..0000000
--- a/core/src/main/java/org/carbondata/query/schema/metadata/DimColumnFilterInfo.java
+++ /dev/null
@@ -1,63 +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.carbondata.query.schema.metadata;
-
-import java.io.Serializable;
-import java.util.List;
-
-public class DimColumnFilterInfo implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 8181578747306832771L;
-
-  private boolean isIncludeFilter;
-
-  private List<Integer> filterList;
-
-  /**
-   * maintain the no dictionary filter values list.
-   */
-  private List<byte[]> noDictionaryFilterValuesList;
-
-  public List<byte[]> getNoDictionaryFilterValuesList() {
-    return noDictionaryFilterValuesList;
-  }
-
-  public boolean isIncludeFilter() {
-    return isIncludeFilter;
-  }
-
-  public void setIncludeFilter(boolean isIncludeFilter) {
-    this.isIncludeFilter = isIncludeFilter;
-  }
-
-  public List<Integer> getFilterList() {
-    return filterList;
-  }
-
-  public void setFilterList(List<Integer> filterList) {
-    this.filterList = filterList;
-  }
-
-  public void setFilterListForNoDictionaryCols(List<byte[]> noDictionaryFilterValuesList) {
-    this.noDictionaryFilterValuesList = noDictionaryFilterValuesList;
-  }
-}


[46/56] [abbrv] incubator-carbondata git commit: fixed GenerateDictionaryExample (#702)

Posted by jb...@apache.org.
fixed GenerateDictionaryExample (#702)



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

Branch: refs/heads/master
Commit: 5c7b043b834687de4c2ca6d1c89be81f8f93fd5e
Parents: 9559a44
Author: Gin-zhj <zh...@huawei.com>
Authored: Mon Jun 20 18:28:33 2016 +0800
Committer: Liang Chen <ch...@huawei.com>
Committed: Mon Jun 20 18:28:33 2016 +0800

----------------------------------------------------------------------
 .../scala/org/carbondata/examples/GenerateDictionaryExample.scala  | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5c7b043b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
index 69e1d2f..2d37bda 100644
--- a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
@@ -65,7 +65,7 @@ object GenerateDictionaryExample {
       lookupRelation1(Option(dataBaseName),
         tableName, None) (carbonContext).asInstanceOf[CarbonRelation]
     val carbonTable = carbonRelation.cubeMeta.carbonTable
-    val dimensions = carbonTable.getDimensionByTableName(tableName)
+    val dimensions = carbonTable.getDimensionByTableName(tableName.toLowerCase())
       .toArray.map(_.asInstanceOf[CarbonDimension])
     // scalastyle:off println
     // print dictionary information


[35/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/logical/NotExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/logical/NotExpression.java b/core/src/main/java/org/carbondata/query/expression/logical/NotExpression.java
deleted file mode 100644
index fc5895d..0000000
--- a/core/src/main/java/org/carbondata/query/expression/logical/NotExpression.java
+++ /dev/null
@@ -1,58 +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.carbondata.query.expression.logical;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.UnaryExpression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class NotExpression extends UnaryExpression {
-  private static final long serialVersionUID = 1L;
-
-  public NotExpression(Expression child) {
-    super(child);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult expResult = child.evaluate(value);
-    expResult.set(DataType.BooleanType, !(expResult.getBoolean()));
-    switch (expResult.getDataType()) {
-      case BooleanType:
-        expResult.set(DataType.BooleanType, !(expResult.getBoolean()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying NOT Expression Filter");
-    }
-    return expResult;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.NOT;
-  }
-
-  @Override public String getString() {
-    return "Not(" + child.getString() + ')';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/logical/OrExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/logical/OrExpression.java b/core/src/main/java/org/carbondata/query/expression/logical/OrExpression.java
deleted file mode 100644
index b5b94ea..0000000
--- a/core/src/main/java/org/carbondata/query/expression/logical/OrExpression.java
+++ /dev/null
@@ -1,60 +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.carbondata.query.expression.logical;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class OrExpression extends BinaryLogicalExpression {
-
-  private static final long serialVersionUID = 4220598043176438380L;
-
-  public OrExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult resultLeft = left.evaluate(value);
-    ExpressionResult resultRight = right.evaluate(value);
-    switch (resultLeft.getDataType()) {
-      case BooleanType:
-        resultLeft.set(DataType.BooleanType, (resultLeft.getBoolean() || resultRight.getBoolean()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying OR Expression Filter");
-    }
-
-    return resultLeft;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.OR;
-  }
-
-  @Override public String getString() {
-    return "Or(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/AndFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/AndFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/AndFilterExecuterImpl.java
deleted file mode 100644
index 22e6eca..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/AndFilterExecuterImpl.java
+++ /dev/null
@@ -1,62 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class AndFilterExecuterImpl implements FilterExecuter {
-
-  private FilterExecuter leftExecuter;
-  private FilterExecuter rightExecuter;
-
-  public AndFilterExecuterImpl(FilterExecuter leftExecuter, FilterExecuter rightExecuter) {
-    this.leftExecuter = leftExecuter;
-    this.rightExecuter = rightExecuter;
-  }
-
-  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
-      throws FilterUnsupportedException {
-    BitSet leftFilters = leftExecuter.applyFilter(blockChunkHolder);
-    if (leftFilters.isEmpty()) {
-      return leftFilters;
-    }
-    BitSet rightFilter = rightExecuter.applyFilter(blockChunkHolder);
-    if (rightFilter.isEmpty()) {
-      return rightFilter;
-    }
-    leftFilters.and(rightFilter);
-    return leftFilters;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
-    if (leftFilters.isEmpty()) {
-      return leftFilters;
-    }
-    BitSet rightFilter = rightExecuter.isScanRequired(blockMaxValue, blockMinValue);
-    if (rightFilter.isEmpty()) {
-      return rightFilter;
-    }
-    leftFilters.and(rightFilter);
-    return leftFilters;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
deleted file mode 100644
index 21847d0..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
+++ /dev/null
@@ -1,190 +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.carbondata.query.filter.executer;
-
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-/**
- * It checks if filter is required on given block and if required, it does
- * linear search on block data and set the bitset.
- */
-public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ColGroupFilterExecuterImpl.class.getName());
-
-  /**
-   * @param dimColResolvedFilterInfo
-   * @param segmentProperties
-   */
-  public ColGroupFilterExecuterImpl(DimColumnResolvedFilterInfo dimColResolvedFilterInfo,
-      SegmentProperties segmentProperties) {
-    super(dimColResolvedFilterInfo, segmentProperties);
-  }
-
-  /**
-   * It fills BitSet with row index which matches filter key
-   */
-  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
-      int numerOfRows) {
-    BitSet bitSet = new BitSet(numerOfRows);
-
-    try {
-      KeyStructureInfo keyStructureInfo = getKeyStructureInfo();
-      byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-      for (int i = 0; i < filterValues.length; i++) {
-        byte[] filterVal = filterValues[i];
-        for (int rowId = 0; rowId < numerOfRows; rowId++) {
-          byte[] colData = new byte[keyStructureInfo.getMaskByteRanges().length];
-          dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, keyStructureInfo);
-          if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) == 0) {
-            bitSet.set(rowId);
-          }
-        }
-      }
-
-    } catch (Exception e) {
-      LOGGER.error(e);
-    }
-
-    return bitSet;
-  }
-
-  /**
-   * It is required for extracting column data from columngroup chunk
-   *
-   * @return
-   * @throws KeyGenException
-   */
-  private KeyStructureInfo getKeyStructureInfo() throws KeyGenException {
-    List<Integer> ordinals = new ArrayList<Integer>();
-    ordinals.add(dimColumnEvaluatorInfo.getColumnIndex());
-
-    int[] maskByteRanges = QueryUtil
-        .getMaskedByteRangeBasedOrdinal(ordinals, segmentProperties.getDimensionKeyGenerator());
-    byte[] maxKey =
-        QueryUtil.getMaxKeyBasedOnOrinal(ordinals, segmentProperties.getDimensionKeyGenerator());
-    int[] maksedByte = QueryUtil
-        .getMaskedByte(segmentProperties.getDimensionKeyGenerator().getKeySizeInBytes(),
-            maskByteRanges);
-    int blockMdkeyStartOffset = QueryUtil.getBlockMdKeyStartOffset(segmentProperties, ordinals);
-
-    KeyStructureInfo restructureInfos = new KeyStructureInfo();
-    restructureInfos.setKeyGenerator(segmentProperties.getDimensionKeyGenerator());
-    restructureInfos.setMaskByteRanges(maskByteRanges);
-    restructureInfos.setMaxKey(maxKey);
-    restructureInfos.setMaskedBytes(maksedByte);
-    restructureInfos.setBlockMdKeyStartOffset(blockMdkeyStartOffset);
-    return restructureInfos;
-  }
-
-  /**
-   * Check if scan is required on given block based on min and max value
-   */
-  public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
-    BitSet bitSet = new BitSet(1);
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    int columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
-    int[] cols = getAllColumns(columnIndex);
-    byte[] maxValue = getMinMaxData(cols, blkMaxVal[blockIndex], columnIndex);
-    byte[] minValue = getMinMaxData(cols, blkMinVal[blockIndex], columnIndex);
-    boolean isScanRequired = false;
-    for (int k = 0; k < filterValues.length; k++) {
-      // filter value should be in range of max and min value i.e
-      // max>filtervalue>min
-      // so filter-max should be negative
-      int maxCompare = ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], maxValue);
-      // and filter-min should be positive
-      int minCompare = ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], minValue);
-
-      // if any filter value is in range than this block needs to be
-      // scanned
-      if (maxCompare <= 0 && minCompare >= 0) {
-        isScanRequired = true;
-        break;
-      }
-    }
-    if (isScanRequired) {
-      bitSet.set(0);
-    }
-    return bitSet;
-  }
-
-  /**
-   * It extract min and max data for given column from stored min max value
-   *
-   * @param cols
-   * @param minMaxData
-   * @param columnIndex
-   * @return
-   */
-  private byte[] getMinMaxData(int[] colGrpColumns, byte[] minMaxData, int columnIndex) {
-    int startIndex = 0;
-    int endIndex = 0;
-    for (int i = 0; i < colGrpColumns.length; i++) {
-      int[] byteRange =
-          segmentProperties.getDimensionKeyGenerator().getKeyByteOffsets(colGrpColumns[i]);
-      int colSize = 0;
-      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
-        colSize++;
-      }
-      if (colGrpColumns[i] == columnIndex) {
-        endIndex = startIndex + colSize;
-        break;
-      }
-      startIndex += colSize;
-    }
-    byte[] data = new byte[endIndex - startIndex];
-    System.arraycopy(minMaxData, startIndex, data, 0, data.length);
-    return data;
-  }
-
-  /**
-   * It returns column groups which have provided column ordinal
-   *
-   * @param columnIndex
-   * @return column group array
-   */
-  private int[] getAllColumns(int columnIndex) {
-    int[][] colGroups = segmentProperties.getColumnGroups();
-    for (int i = 0; i < colGroups.length; i++) {
-      if (QueryUtil.searchInArray(colGroups[i], columnIndex)) {
-        return colGroups[i];
-      }
-    }
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
deleted file mode 100644
index 9de8396..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
+++ /dev/null
@@ -1,206 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.evaluators.DimColumnExecuterFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class ExcludeFilterExecuterImpl implements FilterExecuter {
-
-  DimColumnResolvedFilterInfo dimColEvaluatorInfo;
-  DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-
-  public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo) {
-    this.dimColEvaluatorInfo = dimColEvaluatorInfo;
-  }
-
-  public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
-      KeyGenerator blockKeyGenerator) {
-    this(dimColEvaluatorInfo);
-    dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
-    FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(), blockKeyGenerator,
-        dimColEvaluatorInfo.getDimension(), dimColumnExecuterInfo);
-  }
-
-  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) {
-    if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
-      blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
-          dimColEvaluatorInfo.getColumnIndex());
-    }
-    if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
-      blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()] =
-          blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
-              dimColEvaluatorInfo.getColumnIndex());
-    }
-    return getFilteredIndexes(
-        blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()],
-        blockChunkHolder.getDataBlock().nodeSize());
-  }
-
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimColumnDataChunk, int numerOfRows) {
-    // For high cardinality dimensions.
-    if (dimColumnDataChunk.getAttributes().isNoDictionary()
-        && dimColumnDataChunk instanceof VariableLengthDimensionDataChunk) {
-      return setDirectKeyFilterIndexToBitSet((VariableLengthDimensionDataChunk) dimColumnDataChunk,
-          numerOfRows);
-    }
-    if (null != dimColumnDataChunk.getAttributes().getInvertedIndexes()
-        && dimColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
-      return setFilterdIndexToBitSetWithColumnIndex(
-          (FixedLengthDimensionDataChunk) dimColumnDataChunk, numerOfRows);
-    }
-    return setFilterdIndexToBitSet((FixedLengthDimensionDataChunk) dimColumnDataChunk, numerOfRows);
-  }
-
-  private BitSet setDirectKeyFilterIndexToBitSet(
-      VariableLengthDimensionDataChunk dimColumnDataChunk, int numerOfRows) {
-    BitSet bitSet = new BitSet(numerOfRows);
-    bitSet.flip(0, numerOfRows);
-    List<byte[]> listOfColumnarKeyBlockDataForNoDictionaryVal =
-        dimColumnDataChunk.getCompleteDataChunk();
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    int[] columnIndexArray = dimColumnDataChunk.getAttributes().getInvertedIndexes();
-    int[] columnReverseIndexArray = dimColumnDataChunk.getAttributes().getInvertedIndexesReverse();
-    for (int i = 0; i < filterValues.length; i++) {
-      byte[] filterVal = filterValues[i];
-      if (null != listOfColumnarKeyBlockDataForNoDictionaryVal) {
-
-        if (null != columnReverseIndexArray) {
-          for (int index : columnIndexArray) {
-            byte[] noDictionaryVal =
-                listOfColumnarKeyBlockDataForNoDictionaryVal.get(columnReverseIndexArray[index]);
-            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
-              bitSet.flip(index);
-            }
-          }
-        } else if (null != columnIndexArray) {
-
-          for (int index : columnIndexArray) {
-            byte[] noDictionaryVal =
-                listOfColumnarKeyBlockDataForNoDictionaryVal.get(columnIndexArray[index]);
-            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
-              bitSet.flip(index);
-            }
-          }
-        } else {
-          for (int index = 0;
-               index < listOfColumnarKeyBlockDataForNoDictionaryVal.size(); index++) {
-            if (ByteUtil.UnsafeComparer.INSTANCE
-                .compareTo(filterVal, listOfColumnarKeyBlockDataForNoDictionaryVal.get(index))
-                == 0) {
-              bitSet.flip(index);
-            }
-          }
-
-        }
-
-      }
-    }
-    return bitSet;
-
-  }
-
-  private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      FixedLengthDimensionDataChunk dimColumnDataChunk, int numerOfRows) {
-    int[] columnIndex = dimColumnDataChunk.getAttributes().getInvertedIndexes();
-    int startKey = 0;
-    int last = 0;
-    int startIndex = 0;
-    BitSet bitSet = new BitSet(numerOfRows);
-    bitSet.flip(0, numerOfRows);
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    for (int i = 0; i < filterValues.length; i++) {
-      startKey = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
-      if (startKey == -1) {
-        continue;
-      }
-      bitSet.flip(columnIndex[startKey]);
-      last = startKey;
-      for (int j = startKey + 1; j < numerOfRows; j++) {
-        if (ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(dimColumnDataChunk.getCompleteDataChunk(), j * filterValues[i].length,
-                filterValues[i].length, filterValues[i], 0, filterValues[i].length) == 0) {
-          bitSet.flip(columnIndex[j]);
-          last++;
-        } else {
-          break;
-        }
-      }
-      startIndex = last;
-      if (startIndex >= numerOfRows) {
-        break;
-      }
-    }
-    return bitSet;
-  }
-
-  private BitSet setFilterdIndexToBitSet(FixedLengthDimensionDataChunk dimColumnDataChunk,
-      int numerOfRows) {
-    BitSet bitSet = new BitSet(numerOfRows);
-    int startKey = 0;
-    int last = 0;
-    bitSet.flip(0, numerOfRows);
-    int startIndex = 0;
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    for (int k = 0; k < filterValues.length; k++) {
-      startKey = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[k]);
-      if (startKey == -1) {
-        continue;
-      }
-      bitSet.flip(startKey);
-      last = startKey;
-      for (int j = startKey + 1; j < numerOfRows; j++) {
-        if (ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(dimColumnDataChunk.getCompleteDataChunk(), j * filterValues[k].length,
-                filterValues[k].length, filterValues[k], 0, filterValues[k].length) == 0) {
-          bitSet.flip(j);
-          last++;
-        } else {
-          break;
-        }
-      }
-      startIndex = last;
-      if (startIndex >= numerOfRows) {
-        break;
-      }
-    }
-    return bitSet;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet bitSet = new BitSet(1);
-    bitSet.flip(0, 1);
-    return bitSet;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/FilterExecuter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/FilterExecuter.java b/core/src/main/java/org/carbondata/query/filter/executer/FilterExecuter.java
deleted file mode 100644
index 89ad14e..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/FilterExecuter.java
+++ /dev/null
@@ -1,45 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public interface FilterExecuter {
-
-  /**
-   * API will apply filter based on resolver instance
-   *
-   * @return
-   * @throws FilterUnsupportedException
-   */
-  BitSet applyFilter(BlocksChunkHolder blocksChunkHolder) throws FilterUnsupportedException;
-
-  /**
-   * API will verify whether the block can be shortlisted based on block
-   * max and min key.
-   *
-   * @param blockMaxValue, maximum value of the
-   * @param blockMinValue
-   * @return BitSet
-   */
-  BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue);
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java
deleted file mode 100644
index a15733c..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java
+++ /dev/null
@@ -1,225 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.evaluators.DimColumnExecuterFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class IncludeFilterExecuterImpl implements FilterExecuter {
-
-  protected DimColumnResolvedFilterInfo dimColumnEvaluatorInfo;
-  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-  protected SegmentProperties segmentProperties;
-
-  public IncludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColumnEvaluatorInfo,
-      SegmentProperties segmentProperties) {
-    this.dimColumnEvaluatorInfo = dimColumnEvaluatorInfo;
-    this.segmentProperties = segmentProperties;
-    dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
-    FilterUtil.prepareKeysFromSurrogates(dimColumnEvaluatorInfo.getFilterValues(),
-        segmentProperties.getDimensionKeyGenerator(), dimColumnEvaluatorInfo.getDimension(),
-        dimColumnExecuterInfo);
-
-  }
-
-  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) {
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
-        .get(dimColumnEvaluatorInfo.getColumnIndex());
-    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
-      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
-          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
-    }
-    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
-        blockChunkHolder.getDataBlock().nodeSize());
-  }
-
-  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
-      int numerOfRows) {
-    if (dimensionColumnDataChunk.getAttributes().isNoDictionary()
-        && dimensionColumnDataChunk instanceof VariableLengthDimensionDataChunk) {
-      return setDirectKeyFilterIndexToBitSet(
-          (VariableLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
-    } else if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
-        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
-      return setFilterdIndexToBitSetWithColumnIndex(
-          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
-    }
-
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
-  }
-
-  private BitSet setDirectKeyFilterIndexToBitSet(
-      VariableLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows) {
-    BitSet bitSet = new BitSet(numerOfRows);
-    List<byte[]> listOfColumnarKeyBlockDataForNoDictionaryVals =
-        dimensionColumnDataChunk.getCompleteDataChunk();
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    int[] columnIndexArray = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
-    int[] columnReverseIndexArray =
-        dimensionColumnDataChunk.getAttributes().getInvertedIndexesReverse();
-    for (int i = 0; i < filterValues.length; i++) {
-      byte[] filterVal = filterValues[i];
-      if (null != listOfColumnarKeyBlockDataForNoDictionaryVals) {
-        if (null != columnIndexArray) {
-          for (int index : columnIndexArray) {
-            byte[] noDictionaryVal =
-                listOfColumnarKeyBlockDataForNoDictionaryVals.get(columnReverseIndexArray[index]);
-            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
-              bitSet.set(index);
-            }
-          }
-        } else if (null != columnReverseIndexArray) {
-          for (int index : columnReverseIndexArray) {
-            byte[] noDictionaryVal =
-                listOfColumnarKeyBlockDataForNoDictionaryVals.get(columnReverseIndexArray[index]);
-            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
-              bitSet.set(index);
-            }
-          }
-        } else {
-          for (int index = 0;
-               index < listOfColumnarKeyBlockDataForNoDictionaryVals.size(); index++) {
-            if (ByteUtil.UnsafeComparer.INSTANCE
-                .compareTo(filterVal, listOfColumnarKeyBlockDataForNoDictionaryVals.get(index))
-                == 0) {
-              bitSet.set(index);
-            }
-          }
-        }
-      }
-    }
-    return bitSet;
-
-  }
-
-  private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows) {
-    BitSet bitSet = new BitSet(numerOfRows);
-    int[] columnIndex = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
-    int start = 0;
-    int last = 0;
-    int startIndex = 0;
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    for (int i = 0; i < filterValues.length; i++) {
-      start = CarbonUtil
-          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
-      if (start == -1) {
-        continue;
-      }
-      bitSet.set(columnIndex[start]);
-      last = start;
-      for (int j = start + 1; j < numerOfRows; j++) {
-        if (ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(dimensionColumnDataChunk.getCompleteDataChunk(), j * filterValues[i].length,
-                filterValues[i].length, filterValues[i], 0, filterValues[i].length) == 0) {
-          bitSet.set(columnIndex[j]);
-          last++;
-        } else {
-          break;
-        }
-      }
-      startIndex = last;
-      if (startIndex >= numerOfRows) {
-        break;
-      }
-    }
-    return bitSet;
-  }
-
-  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
-      int numerOfRows) {
-    BitSet bitSet = new BitSet(numerOfRows);
-    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
-      FixedLengthDimensionDataChunk fixedDimensionChunk =
-          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk;
-      int start = 0;
-      int last = 0;
-      int startIndex = 0;
-      byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-      for (int k = 0; k < filterValues.length; k++) {
-        start = CarbonUtil.getFirstIndexUsingBinarySearch(
-            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-            filterValues[k]);
-        if (start == -1) {
-          continue;
-        }
-        bitSet.set(start);
-        last = start;
-        for (int j = start + 1; j < numerOfRows; j++) {
-          if (ByteUtil.UnsafeComparer.INSTANCE
-              .compareTo(fixedDimensionChunk.getCompleteDataChunk(), j * filterValues[k].length,
-                  filterValues[k].length, filterValues[k], 0, filterValues[k].length) == 0) {
-            bitSet.set(j);
-            last++;
-          } else {
-            break;
-          }
-        }
-        startIndex = last;
-        if (startIndex >= numerOfRows) {
-          break;
-        }
-      }
-    }
-    return bitSet;
-  }
-
-  public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
-    BitSet bitSet = new BitSet(1);
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    int columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
-    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
-
-    boolean isScanRequired = false;
-    for (int k = 0; k < filterValues.length; k++) {
-      // filter value should be in range of max and min value i.e
-      // max>filtervalue>min
-      // so filter-max should be negative
-      int maxCompare =
-          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blkMaxVal[blockIndex]);
-      // and filter-min should be positive
-      int minCompare =
-          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blkMinVal[blockIndex]);
-
-      // if any filter value is in range than this block needs to be
-      // scanned
-      if (maxCompare <= 0 && minCompare >= 0) {
-        isScanRequired = true;
-        break;
-      }
-    }
-    if (isScanRequired) {
-      bitSet.set(0);
-    }
-    return bitSet;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/OrFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/OrFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/OrFilterExecuterImpl.java
deleted file mode 100644
index 08b28c1..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/OrFilterExecuterImpl.java
+++ /dev/null
@@ -1,52 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class OrFilterExecuterImpl implements FilterExecuter {
-
-  private FilterExecuter leftExecuter;
-  private FilterExecuter rightExecuter;
-
-  public OrFilterExecuterImpl(FilterExecuter leftExecuter, FilterExecuter rightExecuter) {
-    this.leftExecuter = leftExecuter;
-    this.rightExecuter = rightExecuter;
-  }
-
-  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
-      throws FilterUnsupportedException {
-    BitSet leftFilters = leftExecuter.applyFilter(blockChunkHolder);
-    BitSet rightFilters = rightExecuter.applyFilter(blockChunkHolder);
-    leftFilters.or(rightFilters);
-
-    return leftFilters;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
-    BitSet rightFilters = rightExecuter.isScanRequired(blockMaxValue, blockMinValue);
-    leftFilters.or(rightFilters);
-    return leftFilters;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/RestructureFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RestructureFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/RestructureFilterExecuterImpl.java
deleted file mode 100644
index 9f1c5ca..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/RestructureFilterExecuterImpl.java
+++ /dev/null
@@ -1,55 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.evaluators.DimColumnExecuterFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class RestructureFilterExecuterImpl implements FilterExecuter {
-
-  DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-
-  public RestructureFilterExecuterImpl(DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo,
-      KeyGenerator blockKeyGenerator) {
-    dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
-    FilterUtil
-        .prepareKeysFromSurrogates(dimColumnResolvedFilterInfo.getFilterValues(), blockKeyGenerator,
-            dimColumnResolvedFilterInfo.getDimension(), dimColumnExecuterInfo);
-  }
-
-  @Override public BitSet applyFilter(BlocksChunkHolder blocksChunkHolder) {
-    BitSet bitSet = new BitSet(blocksChunkHolder.getDataBlock().nodeSize());
-    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
-    if (null != filterValues && filterValues.length > 0) {
-      bitSet.set(0, blocksChunkHolder.getDataBlock().nodeSize());
-    }
-    return bitSet;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet bitSet = new BitSet(1);
-    bitSet.set(0);
-    return bitSet;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java
deleted file mode 100644
index 0377580..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java
+++ /dev/null
@@ -1,331 +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.carbondata.query.filter.executer;
-
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbonfilterinterface.RowImpl;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.complex.querytypes.GenericQueryType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class RowLevelFilterExecuterImpl implements FilterExecuter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName());
-  protected List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
-  protected List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
-  protected Expression exp;
-  protected AbsoluteTableIdentifier tableIdentifier;
-
-  public RowLevelFilterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
-      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier) {
-    this.dimColEvaluatorInfoList = dimColEvaluatorInfoList;
-    if (null == msrColEvalutorInfoList) {
-      this.msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(20);
-    } else {
-      this.msrColEvalutorInfoList = msrColEvalutorInfoList;
-    }
-    this.exp = exp;
-    this.tableIdentifier = tableIdentifier;
-  }
-
-  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
-      throws FilterUnsupportedException {
-    for (DimColumnResolvedFilterInfo dimColumnEvaluatorInfo : dimColEvaluatorInfoList) {
-      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.ARRAY
-          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.STRUCT) {
-        if (null == blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo
-            .getColumnIndex()]) {
-          blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()] =
-              blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
-                  dimColumnEvaluatorInfo.getColumnIndex());
-        }
-      } else {
-        GenericQueryType complexType = dimColumnEvaluatorInfo.getComplexTypesWithBlockStartIndex()
-            .get(dimColumnEvaluatorInfo.getColumnIndex());
-        complexType.fillRequiredBlockData(blockChunkHolder);
-      }
-    }
-
-    // CHECKSTYLE:OFF Approval No:Approval-V1R2C10_001
-    if (null != msrColEvalutorInfoList) {
-      for (MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo : msrColEvalutorInfoList) {
-        if (msrColumnEvalutorInfo.isMeasureExistsInCurrentSlice() && null == blockChunkHolder
-            .getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]) {
-          blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()] =
-              blockChunkHolder.getDataBlock().getMeasureChunk(blockChunkHolder.getFileReader(),
-                  msrColumnEvalutorInfo.getColumnIndex());
-        }
-      }
-    }
-    // CHECKSTYLE:ON
-
-    int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
-    BitSet set = new BitSet(numberOfRows);
-    RowIntf row = new RowImpl();
-
-    // CHECKSTYLE:OFF Approval No:Approval-V1R2C10_007
-    for (int index = 0; index < numberOfRows; index++) {
-      try {
-        createRow(blockChunkHolder, row, index);
-      } catch (QueryExecutionException e1) {
-        // TODO Auto-generated catch block
-        e1.printStackTrace();
-      }
-      try {
-        Boolean rslt = exp.evaluate(row).getBoolean();
-        if (null != rslt && rslt) {
-          set.set(index);
-        }
-      } catch (FilterUnsupportedException e) {
-        throw new FilterUnsupportedException(e.getMessage());
-      }
-    }
-    // CHECKSTYLE:ON
-
-    return set;
-  }
-
-  /**
-   * Method will read the members of particular dimension block and create
-   * a row instance for further processing of the filters
-   *
-   * @param blockChunkHolder
-   * @param row
-   * @param index
-   * @throws QueryExecutionException
-   */
-  private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int index)
-      throws QueryExecutionException {
-    Object[] record = new Object[dimColEvaluatorInfoList.size() + msrColEvalutorInfoList.size()];
-    String memberString = null;
-    for (DimColumnResolvedFilterInfo dimColumnEvaluatorInfo : dimColEvaluatorInfoList) {
-      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.ARRAY
-          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.STRUCT) {
-        if (!dimColumnEvaluatorInfo.isDimensionExistsInCurrentSilce()) {
-          record[dimColumnEvaluatorInfo.getRowIndex()] = dimColumnEvaluatorInfo.getDefaultValue();
-        }
-        if (!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
-            && blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo
-            .getColumnIndex()] instanceof VariableLengthDimensionDataChunk) {
-
-          VariableLengthDimensionDataChunk dimensionColumnDataChunk =
-              (VariableLengthDimensionDataChunk) blockChunkHolder
-                  .getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()];
-          if (null != dimensionColumnDataChunk.getCompleteDataChunk()) {
-            memberString =
-                readMemberBasedOnNoDictionaryVal(dimColumnEvaluatorInfo, dimensionColumnDataChunk,
-                    index);
-            if (null != memberString) {
-              if (memberString.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
-                memberString = null;
-              }
-            }
-            record[dimColumnEvaluatorInfo.getRowIndex()] = DataTypeUtil
-                .getDataBasedOnDataType(memberString,
-                    dimColumnEvaluatorInfo.getDimension().getDataType());
-          } else {
-            continue;
-          }
-        } else {
-          int dictionaryValue =
-              readSurrogatesFromColumnBlock(blockChunkHolder, index, dimColumnEvaluatorInfo);
-          Dictionary forwardDictionary = null;
-          if (dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
-              && !dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-            memberString =
-                getFilterActualValueFromDictionaryValue(dimColumnEvaluatorInfo, dictionaryValue,
-                    forwardDictionary);
-            record[dimColumnEvaluatorInfo.getRowIndex()] = DataTypeUtil
-                .getDataBasedOnDataType(memberString,
-                    dimColumnEvaluatorInfo.getDimension().getDataType());
-          } else if (dimColumnEvaluatorInfo.getDimension()
-              .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-
-            Object member = getFilterActualValueFromDirectDictionaryValue(dimColumnEvaluatorInfo,
-                dictionaryValue);
-            record[dimColumnEvaluatorInfo.getRowIndex()] = member;
-          }
-        }
-      }
-    }
-
-    DataType msrType;
-
-    for (MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo : msrColEvalutorInfoList) {
-      switch (msrColumnEvalutorInfo.getType()) {
-        case LONG:
-          msrType = DataType.LONG;
-          break;
-        case DECIMAL:
-          msrType = DataType.DECIMAL;
-          break;
-        default:
-          msrType = DataType.DOUBLE;
-      }
-      // if measure doesnt exist then set the default value.
-      if (!msrColumnEvalutorInfo.isMeasureExistsInCurrentSlice()) {
-        record[msrColumnEvalutorInfo.getRowIndex()] = msrColumnEvalutorInfo.getDefaultValue();
-      } else {
-        Object msrValue;
-        switch (msrType) {
-          case LONG:
-            msrValue =
-                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
-                    .getMeasureDataHolder().getReadableLongValueByIndex(index);
-            break;
-          case DECIMAL:
-            msrValue =
-                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
-                    .getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-            break;
-          default:
-            msrValue =
-                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
-                    .getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-        }
-        record[msrColumnEvalutorInfo.getRowIndex()] = msrValue;
-
-      }
-    }
-    row.setValues(record);
-  }
-
-  /**
-   * method will read the actual data from the direct dictionary generator
-   * by passing direct dictionary value.
-   *
-   * @param dimColumnEvaluatorInfo
-   * @param dictionaryValue
-   * @return
-   */
-  private Object getFilterActualValueFromDirectDictionaryValue(
-      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int dictionaryValue) {
-    Object memberString = null;
-    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-        .getDirectDictionaryGenerator(dimColumnEvaluatorInfo.getDimension().getDataType());
-    if (null != directDictionaryGenerator) {
-      memberString = directDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
-    }
-    return memberString;
-  }
-
-  /**
-   * Read the actual filter member by passing the dictionary value from
-   * the forward dictionary cache which which holds column wise cache
-   *
-   * @param dimColumnEvaluatorInfo
-   * @param dictionaryValue
-   * @param forwardDictionary
-   * @return
-   * @throws QueryExecutionException
-   */
-  private String getFilterActualValueFromDictionaryValue(
-      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int dictionaryValue,
-      Dictionary forwardDictionary) throws QueryExecutionException {
-    String memberString;
-    try {
-      forwardDictionary = FilterUtil
-          .getForwardDictionaryCache(tableIdentifier, dimColumnEvaluatorInfo.getDimension());
-    } catch (QueryExecutionException e) {
-      throw new QueryExecutionException(e);
-    }
-
-    memberString = forwardDictionary.getDictionaryValueForKey(dictionaryValue);
-    if (null != memberString) {
-      if (memberString.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
-        memberString = null;
-      }
-    }
-    return memberString;
-  }
-
-  /**
-   * read the filter member dictionary data from the block corresponding to
-   * applied filter column
-   *
-   * @param blockChunkHolder
-   * @param index
-   * @param dimColumnEvaluatorInfo
-   * @return
-   */
-  private int readSurrogatesFromColumnBlock(BlocksChunkHolder blockChunkHolder, int index,
-      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo) {
-    byte[] rawData =
-        blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()]
-            .getChunkData(index);
-    ByteBuffer byteBuffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
-    int dictionaryValue = CarbonUtil.getSurrogateKey(rawData, byteBuffer);
-    return dictionaryValue;
-  }
-
-  /**
-   * Reading the blocks for no dictionary data, in no dictionary case
-   * directly the filter data will read, no need to scan the dictionary
-   * or read the dictionary value.
-   *
-   * @param dimColumnEvaluatorInfo
-   * @param dimensionColumnDataChunk
-   * @param index
-   * @return
-   */
-  private String readMemberBasedOnNoDictionaryVal(
-      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo,
-      VariableLengthDimensionDataChunk dimensionColumnDataChunk, int index) {
-    byte[] noDictionaryVals;
-    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexesReverse()) {
-      // Getting the data for direct surrogates.
-      noDictionaryVals = dimensionColumnDataChunk.getCompleteDataChunk()
-          .get(dimensionColumnDataChunk.getAttributes().getInvertedIndexesReverse()[index]);
-    } else {
-      noDictionaryVals = dimensionColumnDataChunk.getCompleteDataChunk().get(index);
-    }
-    return new String(noDictionaryVals, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet bitSet = new BitSet(1);
-    bitSet.set(0);
-    return bitSet;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
deleted file mode 100644
index 88290d9..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-
-public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
-  private byte[][] filterRangeValues;
-
-  public RowLevelRangeGrtThanFiterExecuterImpl(
-      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
-      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
-    this.filterRangeValues = filterRangeValues;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet bitSet = new BitSet(1);
-    byte[][] filterValues = this.filterRangeValues;
-    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
-    boolean isScanRequired = false;
-    for (int k = 0; k < filterValues.length; k++) {
-      // filter value should be in range of max and min value i.e
-      // max>filtervalue>min
-      // so filter-max should be negative
-      int maxCompare =
-          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMaxValue[columnIndex]);
-
-      // if any filter value is in range than this block needs to be
-      // scanned means always less than block max range.
-      if (maxCompare < 0) {
-        isScanRequired = true;
-        break;
-      }
-    }
-    if (isScanRequired) {
-      bitSet.set(0);
-    }
-    return bitSet;
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
deleted file mode 100644
index 4d7d962..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-
-public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl {
-
-  private byte[][] filterRangeValues;
-
-  public RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
-      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
-      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
-    this.filterRangeValues = filterRangeValues;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet bitSet = new BitSet(1);
-    byte[][] filterValues = this.filterRangeValues;
-    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
-    boolean isScanRequired = false;
-    for (int k = 0; k < filterValues.length; k++) {
-      // filter value should be in range of max and min value i.e
-      // max>filtervalue>min
-      // so filter-max should be negative
-      int maxCompare =
-          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMaxValue[columnIndex]);
-      // if any filter value is in range than this block needs to be
-      // scanned less than equal to max range.
-      if (maxCompare <= 0) {
-        isScanRequired = true;
-        break;
-      }
-    }
-    if (isScanRequired) {
-      bitSet.set(0);
-    }
-    return bitSet;
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
deleted file mode 100644
index e5f10ef..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-
-public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilterExecuterImpl {
-  private byte[][] filterRangeValues;
-
-  public RowLevelRangeLessThanEqualFilterExecuterImpl(
-      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
-      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
-    this.filterRangeValues = filterRangeValues;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet bitSet = new BitSet(1);
-    byte[][] filterValues = this.filterRangeValues;
-    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
-    boolean isScanRequired = false;
-    for (int k = 0; k < filterValues.length; k++) {
-      // and filter-min should be positive
-      int minCompare =
-          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMinValue[columnIndex]);
-
-      // if any filter applied is not in range of min and max of block
-      // then since its a less than equal to fiter validate whether the block
-      // min range is less than equal to applied filter member
-      if (minCompare >= 0) {
-        isScanRequired = true;
-        break;
-      }
-    }
-    if (isScanRequired) {
-      bitSet.set(0);
-    }
-    return bitSet;
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
deleted file mode 100644
index 56f7393..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ /dev/null
@@ -1,65 +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.carbondata.query.filter.executer;
-
-import java.util.BitSet;
-import java.util.List;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-
-public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
-  private byte[][] filterRangeValues;
-
-  public RowLevelRangeLessThanFiterExecuterImpl(
-      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
-      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
-    this.filterRangeValues = filterRangeValues;
-  }
-
-  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
-    BitSet bitSet = new BitSet(1);
-    byte[][] filterValues = this.filterRangeValues;
-    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
-    boolean isScanRequired = false;
-    for (int k = 0; k < filterValues.length; k++) {
-      // and filter-min should be positive
-      int minCompare =
-          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMinValue[columnIndex]);
-
-      // if any filter applied is not in range of min and max of block
-      // then since its a less than fiter validate whether the block
-      // min range is less  than applied filter member
-      if (minCompare > 0) {
-        isScanRequired = true;
-        break;
-      }
-    }
-    if (isScanRequired) {
-      bitSet.set(0);
-    }
-    return bitSet;
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeTypeExecuterFacory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeTypeExecuterFacory.java b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeTypeExecuterFacory.java
deleted file mode 100644
index 5a5f2d2..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeTypeExecuterFacory.java
+++ /dev/null
@@ -1,90 +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.carbondata.query.filter.executer;
-
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.filter.resolver.FilterResolverIntf;
-import org.carbondata.query.filter.resolver.RowLevelRangeFilterResolverImpl;
-
-public class RowLevelRangeTypeExecuterFacory {
-
-  private RowLevelRangeTypeExecuterFacory() {
-
-  }
-
-  /**
-   * The method returns the Row Level Range fiter type instance based on
-   * filter tree resolver type.
-   *
-   * @param filterExpressionResolverTree
-   * @param dataType                     DataType
-   * @return the generator instance
-   */
-  public static RowLevelFilterExecuterImpl getRowLevelRangeTypeExecuter(
-      FilterExecuterType filterExecuterType, FilterResolverIntf filterExpressionResolverTree) {
-    switch (filterExecuterType) {
-
-      case ROWLEVEL_LESSTHAN:
-        return new RowLevelRangeLessThanFiterExecuterImpl(
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getDimColEvaluatorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getMsrColEvalutorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
-      case ROWLEVEL_LESSTHAN_EQUALTO:
-        return new RowLevelRangeLessThanEqualFilterExecuterImpl(
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getDimColEvaluatorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getMsrColEvalutorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
-      case ROWLEVEL_GREATERTHAN_EQUALTO:
-        return new RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getDimColEvaluatorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getMsrColEvalutorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
-      case ROWLEVEL_GREATERTHAN:
-        return new RowLevelRangeGrtThanFiterExecuterImpl(
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getDimColEvaluatorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getMsrColEvalutorInfoList(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
-            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
-      default:
-        // Scenario wont come logic must break
-        return null;
-
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/AndFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/AndFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/AndFilterResolverImpl.java
deleted file mode 100644
index 1bebae2..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/AndFilterResolverImpl.java
+++ /dev/null
@@ -1,51 +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.carbondata.query.filter.resolver;
-
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-
-public class AndFilterResolverImpl extends LogicalFilterResolverImpl {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = -761688076874662001L;
-
-  public AndFilterResolverImpl(FilterResolverIntf leftEvalutor, FilterResolverIntf rightEvalutor,
-      ExpressionType filterExpressionType) {
-    super(leftEvalutor, rightEvalutor, filterExpressionType);
-  }
-
-  @Override public void getStartKey(SegmentProperties segmentProperties, long[] startKeys,
-      SortedMap<Integer, byte[]> noDicStartKeys) {
-    leftEvalutor.getStartKey(segmentProperties, startKeys, noDicStartKeys);
-    rightEvalutor.getStartKey(segmentProperties, startKeys, noDicStartKeys);
-  }
-
-  @Override public void getEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> noDicEndKeys) {
-    leftEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
-    rightEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
-  }
-}


[12/56] [abbrv] incubator-carbondata git commit: Revert "[Issue - 626] csv file having some rows not in proper format is not treating them to the bad records." (#687)

Posted by jb...@apache.org.
Revert "[Issue - 626] csv file having some rows not in proper format is not treating them to the bad records." (#687)



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

Branch: refs/heads/master
Commit: 6c2469db288a7df59284c7a13aaed1d0b417edef
Parents: 075dd92
Author: Venkata Ramana G <g....@gmail.com>
Authored: Fri Jun 17 22:58:00 2016 +0530
Committer: GitHub <no...@github.com>
Committed: Fri Jun 17 22:58:00 2016 +0530

----------------------------------------------------------------------
 .../src/test/resources/InvalidCsvFormatdata.csv |   3 -
 .../csvreaderstep/BlockDataHandler.java         | 576 +++++++++----------
 .../processing/csvreaderstep/CsvInput.java      |  10 +-
 .../processing/csvreaderstep/CsvInputMeta.java  |  66 +--
 .../graphgenerator/GraphGenerator.java          |   3 +-
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |  22 +-
 .../util/CarbonDataProcessorUtil.java           |  34 +-
 7 files changed, 318 insertions(+), 396 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6c2469db/integration/spark/src/test/resources/InvalidCsvFormatdata.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/InvalidCsvFormatdata.csv b/integration/spark/src/test/resources/InvalidCsvFormatdata.csv
deleted file mode 100644
index d61dd50..0000000
--- a/integration/spark/src/test/resources/InvalidCsvFormatdata.csv
+++ /dev/null
@@ -1,3 +0,0 @@
-ID,date,country,name,phonetype,serialname,salary
-1,2015/7/23,china,aaa1,phone197,ASD69643,15000
-2
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6c2469db/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java b/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
index d832504..29b4a54 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
@@ -27,8 +27,6 @@ import java.util.List;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.load.BlockDetails;
-import org.carbondata.processing.surrogatekeysgenerator.csvbased.BadRecordslogger;
-import org.carbondata.processing.util.CarbonDataProcessorUtil;
 
 import org.apache.commons.vfs.FileObject;
 import org.pentaho.di.core.exception.KettleConversionException;
@@ -54,21 +52,19 @@ public class BlockDataHandler {
   public int bufferSize;
   public long bytesToSkipInFirstFile;
   public long totalBytesRead;
-  public CsvInputMeta meta;
-  public CsvInputData data;
+  public CsvInputMeta meta ;
+  public CsvInputData data ;
   public TransMeta transMeta;
   public boolean isNeedToSkipFirstLineInBlock;
   public long currentOffset;
 
   protected InputStream bufferedInputStream;
-  protected BadRecordslogger badRecordslogger;
-  private String badRecordFileName;
+
 
   public BlockDataHandler() {
     byteBuffer = new byte[] {};
     isNeedToSkipFirstLineInBlock = true;
     currentOffset = 0;
-
   }
 
   // Resize
@@ -202,12 +198,7 @@ public class BlockDataHandler {
 
       // Don't skip again in the next file...
       this.bytesToSkipInFirstFile = -1L;
-      String key = meta.getDatabaseName() + '/' + meta.getTableName() + '_' + meta.getTableName();
-      badRecordFileName = transMeta.getVariable("csvInputFilePath");
-      badRecordFileName = null != badRecordFileName ? badRecordFileName : meta.getTableName();
-      badRecordFileName = CarbonDataProcessorUtil.getBagLogFileName(badRecordFileName);
-      badRecordslogger = new BadRecordslogger(key, badRecordFileName, CarbonDataProcessorUtil
-          .getBadLogStoreLocation(meta.getDatabaseName() + '/' + meta.getTableName()));
+
       return true;
     } catch (KettleException e) {
       throw e;
@@ -215,12 +206,11 @@ public class BlockDataHandler {
       throw new KettleException(e);
     }
   }
-
   protected void initializeFileReader(FileObject fileObject) throws IOException {
     //using file object to get path can return a valid path which for new inputstream
     String filePath = KettleVFS.getFilename(fileObject);
-    this.bufferedInputStream = FileFactory
-        .getDataInputStream(filePath, FileFactory.getFileType(filePath), data.preferredBufferSize);
+    this.bufferedInputStream = FileFactory.getDataInputStream(filePath,
+        FileFactory.getFileType(filePath), data.preferredBufferSize);
     //when open a new file, need to initialize all info
     this.byteBuffer = new byte[data.preferredBufferSize];
     this.bufferSize = 0;
@@ -228,16 +218,14 @@ public class BlockDataHandler {
     this.endBuffer = 0;
     this.currentOffset = 0;
   }
-
   /**
-   * skip the offset and reset the value
-   *
+   *  skip the offset and reset the value
    * @param filePath
    * @param offset
    * @throws IOException
    */
-  protected void initializeFileReader(String filePath, long offset)
-      throws IOException, KettleFileException {
+  protected void initializeFileReader(String filePath,long offset) throws IOException,
+          KettleFileException {
     if (this.bufferedInputStream != null) {
       this.bufferedInputStream.close();
     }
@@ -248,9 +236,8 @@ public class BlockDataHandler {
       filePath = KettleVFS.getFilename(fileObject);
     }
 
-    this.bufferedInputStream = FileFactory
-        .getDataInputStream(filePath, FileFactory.getFileType(filePath), data.preferredBufferSize,
-            offset);
+    this.bufferedInputStream = FileFactory.getDataInputStream(filePath,
+        FileFactory.getFileType(filePath), data.preferredBufferSize,offset);
     this.byteBuffer = new byte[data.preferredBufferSize];
     this.bufferSize = 0;
     this.startBuffer = 0;
@@ -268,334 +255,337 @@ public class BlockDataHandler {
   public Object[] readOneRow(boolean doConversions) throws KettleException {
 
     try {
-      while (true) {
-        Object[] outputRowData =
-            RowDataUtil.allocateRowData(data.outputRowMeta.size() - RowDataUtil.OVER_ALLOCATE_SIZE);
-        int outputIndex = 0;
-        boolean newLineFound = false;
-        boolean endOfBuffer = false;
-        int newLines = 0;
-        List<Exception> conversionExceptions = null;
-        List<ValueMetaInterface> exceptionFields = null;
-
-        // The strategy is as follows...
-        // We read a block of byte[] from the file.
-        // We scan for the separators in the file (NOT for line feeds etc)
-        // Then we scan that block of data.
-        // We keep a byte[] that we extend if needed..
-        // At the end of the block we read another, etc.
-        //
-        // Let's start by looking where we left off reading.
-        //
-        while (!newLineFound && outputIndex < meta.getInputFields().length) {
-
-          if (checkBufferSize() && outputRowData != null) {
-            // Last row was being discarded if the last item is null and
-            // there is no end of line delimiter
-            //if (outputRowData != null) {
-            // Make certain that at least one record exists before
-            // filling the rest of them with null
-            if (outputIndex > 0) {
-              return (outputRowData);
-            }
 
-            return null; // nothing more to read, call it a day.
+      Object[] outputRowData = RowDataUtil
+          .allocateRowData(data.outputRowMeta.size() - RowDataUtil.OVER_ALLOCATE_SIZE);
+      int outputIndex = 0;
+      boolean newLineFound = false;
+      boolean endOfBuffer = false;
+      int newLines = 0;
+      List<Exception> conversionExceptions = null;
+      List<ValueMetaInterface> exceptionFields = null;
+
+      // The strategy is as follows...
+      // We read a block of byte[] from the file.
+      // We scan for the separators in the file (NOT for line feeds etc)
+      // Then we scan that block of data.
+      // We keep a byte[] that we extend if needed..
+      // At the end of the block we read another, etc.
+      //
+      // Let's start by looking where we left off reading.
+      //
+      while (!newLineFound && outputIndex < meta.getInputFields().length) {
+
+        if (checkBufferSize() && outputRowData != null) {
+          // Last row was being discarded if the last item is null and
+          // there is no end of line delimiter
+          //if (outputRowData != null) {
+          // Make certain that at least one record exists before
+          // filling the rest of them with null
+          if (outputIndex > 0) {
+            return (outputRowData);
           }
 
-          // OK, at this point we should have data in the byteBuffer and we should be able
-          // to scan for the next
-          // delimiter (;)
-          // So let's look for a delimiter.
-          // Also skip over the enclosures ("), it is NOT taking into account
-          // escaped enclosures.
-          // Later we can add an option for having escaped or double enclosures
-          // in the file. <sigh>
+          return null; // nothing more to read, call it a day.
+        }
+
+        // OK, at this point we should have data in the byteBuffer and we should be able
+        // to scan for the next
+        // delimiter (;)
+        // So let's look for a delimiter.
+        // Also skip over the enclosures ("), it is NOT taking into account
+        // escaped enclosures.
+        // Later we can add an option for having escaped or double enclosures
+        // in the file. <sigh>
+        //
+        boolean delimiterFound = false;
+        boolean enclosureFound = false;
+        int escapedEnclosureFound = 0;
+        while (!delimiterFound) {
+          // If we find the first char, we might find others as well ;-)
+          // Single byte delimiters only for now.
           //
-          boolean delimiterFound = false;
-          boolean enclosureFound = false;
-          int escapedEnclosureFound = 0;
-          while (!delimiterFound) {
-            // If we find the first char, we might find others as well ;-)
-            // Single byte delimiters only for now.
-            //
-            if (data.delimiterMatcher
-                .matchesPattern(this.byteBuffer, this.endBuffer, data.delimiter)) {
-              delimiterFound = true;
+          if (data.delimiterMatcher
+              .matchesPattern(this.byteBuffer, this.endBuffer, data.delimiter)) {
+            delimiterFound = true;
+          }
+          // Perhaps we found a (pre-mature) new line?
+          //
+          else if (
+              // In case we are not using an enclosure and in case fields contain new
+              // lines we need to make sure that we check the newlines possible flag.
+              // If the flag is enable we skip newline checking except for the last field
+              // in the row. In that one we can't support newlines without
+              // enclosure (handled below).
+              //
+              (!meta.isNewlinePossibleInFields()
+                  || outputIndex == meta.getInputFields().length - 1) && (
+                  data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer)
+                      || data.crLfMatcher
+                      .isLineFeed(this.byteBuffer, this.endBuffer))) {
+
+            if (data.encodingType.equals(EncodingType.DOUBLE_LITTLE_ENDIAN)
+                || data.encodingType.equals(EncodingType.DOUBLE_BIG_ENDIAN)) {
+              this.endBuffer += 2;
+              this.currentOffset +=2;
+            } else {
+              this.endBuffer++;
+              this.currentOffset++;
             }
-            // Perhaps we found a (pre-mature) new line?
-            //
-            else if (
-                // In case we are not using an enclosure and in case fields contain new
-                // lines we need to make sure that we check the newlines possible flag.
-                // If the flag is enable we skip newline checking except for the last field
-                // in the row. In that one we can't support newlines without
-                // enclosure (handled below).
-                (!meta.isNewlinePossibleInFields()
-                    || outputIndex == meta.getInputFields().length - 1) && (
-                    data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer) || data.crLfMatcher
-                        .isLineFeed(this.byteBuffer, this.endBuffer))) {
-
-              if (data.encodingType.equals(EncodingType.DOUBLE_LITTLE_ENDIAN) || data.encodingType
-                  .equals(EncodingType.DOUBLE_BIG_ENDIAN)) {
-                this.endBuffer += 2;
-                this.currentOffset += 2;
-              } else {
-                this.endBuffer++;
-                this.currentOffset++;
-              }
 
-              this.totalBytesRead++;
-              newLines = 1;
+            this.totalBytesRead++;
+            newLines = 1;
+
+            if (this.endBuffer >= this.bufferSize) {
+              // Oops, we need to read more data...
+              // Better resize this before we read other things in it...
+              //
+              this.resizeByteBufferArray();
+
+              // Also read another chunk of data, now that we have the space for it...
+              // Ignore EOF, there might be other stuff in the buffer.
+              //
+              this.readBufferFromFile();
+            }
 
+            // re-check for double delimiters...
+            if (data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer)
+                || data.crLfMatcher.isLineFeed(this.byteBuffer, this.endBuffer)) {
+              this.endBuffer++;
+              this.currentOffset++;
+              this.totalBytesRead++;
+              newLines = 2;
               if (this.endBuffer >= this.bufferSize) {
                 // Oops, we need to read more data...
                 // Better resize this before we read other things in it...
                 //
                 this.resizeByteBufferArray();
 
-                // Also read another chunk of data, now that we have the space for it...
+                // Also read another chunk of data, now that we have the space for
+                // it...
                 // Ignore EOF, there might be other stuff in the buffer.
                 //
                 this.readBufferFromFile();
               }
+            }
 
-              // re-check for double delimiters...
-              if (data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer) || data.crLfMatcher
-                  .isLineFeed(this.byteBuffer, this.endBuffer)) {
-                this.endBuffer++;
-                this.currentOffset++;
-                this.totalBytesRead++;
-                newLines = 2;
-                if (this.endBuffer >= this.bufferSize) {
-                  // Oops, we need to read more data...
-                  // Better resize this before we read other things in it...
-                  //
-                  this.resizeByteBufferArray();
-
-                  // Also read another chunk of data, now that we have the space for
-                  // it...
-                  // Ignore EOF, there might be other stuff in the buffer.
-                  //
-                  this.readBufferFromFile();
-                }
+            newLineFound = true;
+            delimiterFound = true;
+          }
+          // Perhaps we need to skip over an enclosed part?
+          // We always expect exactly one enclosure character
+          // If we find the enclosure doubled, we consider it escaped.
+          // --> "" is converted to " later on.
+          //
+          else if (data.enclosure != null && data.enclosureMatcher
+              .matchesPattern(this.byteBuffer, this.endBuffer, data.enclosure)) {
+
+            enclosureFound = true;
+            boolean keepGoing;
+            do {
+              if (this.increaseEndBuffer()) {
+                enclosureFound = false;
+                break;
               }
 
-              newLineFound = true;
-              delimiterFound = true;
-            }
-            // Perhaps we need to skip over an enclosed part?
-            // We always expect exactly one enclosure character
-            // If we find the enclosure doubled, we consider it escaped.
-            // --> "" is converted to " later on.
-            //
-            else if (data.enclosure != null && data.enclosureMatcher
-                .matchesPattern(this.byteBuffer, this.endBuffer, data.enclosure)) {
+              if (!doConversions) {
+                //when catch the block which need to skip first line
+                //the complete row like: abc,"cdf","efg",hij
+                //but this row is split to different blocks
+                //in this block,the remaining row like :  fg",hij
+                //so if we meet the enclosure in the skip line, when we meet \r or \n ,let's break
+                if (data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer)
+                        || data.crLfMatcher.isLineFeed(this.byteBuffer, this.endBuffer)) {
+                  enclosureFound = false;
+                  break;
+                }
+              }
 
-              enclosureFound = true;
-              boolean keepGoing;
-              do {
+              keepGoing = !data.enclosureMatcher
+                  .matchesPattern(this.byteBuffer, this.endBuffer,
+                      data.enclosure);
+              if (!keepGoing) {
+                // We found an enclosure character.
+                // Read another byte...
                 if (this.increaseEndBuffer()) {
                   enclosureFound = false;
                   break;
                 }
 
-                if (!doConversions) {
-                  //when catch the block which need to skip first line
-                  //the complete row like: abc,"cdf","efg",hij
-                  //but this row is split to different blocks
-                  //in this block,the remaining row like :  fg",hij
-                  //so if we meet the enclosure in the skip line, when we meet \r or \n ,let's break
-                  if (data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer) || data.crLfMatcher
-                      .isLineFeed(this.byteBuffer, this.endBuffer)) {
-                    enclosureFound = false;
-                    break;
-                  }
+                // If this character is also an enclosure, we can consider the
+                // enclosure "escaped".
+                // As such, if this is an enclosure, we keep going...
+                //
+                keepGoing = data.enclosureMatcher
+                    .matchesPattern(this.byteBuffer, this.endBuffer,
+                        data.enclosure);
+                if (keepGoing) {
+                  escapedEnclosureFound++;
+                } else {
+                  /**
+                   * <pre>
+                   * fix for customer issue.
+                   * after last enclosure there must be either field end or row
+                   * end otherwise enclosure is field content.
+                   * Example:
+                   * EMPNAME, COMPANY
+                   * 'emp'aa','comab'
+                   * 'empbb','com'cd'
+                   * Here enclosure after emp(emp') and after com(com')
+                   * are not the last enclosures
+                   * </pre>
+                   */
+                  keepGoing = !(data.delimiterMatcher
+                      .matchesPattern(this.byteBuffer, this.endBuffer,
+                          data.delimiter) || data.crLfMatcher
+                      .isReturn(this.byteBuffer, this.endBuffer)
+                      || data.crLfMatcher
+                      .isLineFeed(this.byteBuffer, this.endBuffer));
                 }
 
-                keepGoing = !data.enclosureMatcher
-                    .matchesPattern(this.byteBuffer, this.endBuffer, data.enclosure);
-                if (!keepGoing) {
-                  // We found an enclosure character.
-                  // Read another byte...
-                  if (this.increaseEndBuffer()) {
-                    enclosureFound = false;
-                    break;
-                  }
-
-                  // If this character is also an enclosure, we can consider the
-                  // enclosure "escaped".
-                  // As such, if this is an enclosure, we keep going...
-                  //
-                  keepGoing = data.enclosureMatcher
-                      .matchesPattern(this.byteBuffer, this.endBuffer, data.enclosure);
-                  if (keepGoing) {
-                    escapedEnclosureFound++;
-                  } else {
-                    /**
-                     * <pre>
-                     * fix for customer issue.
-                     * after last enclosure there must be either field end or row
-                     * end otherwise enclosure is field content.
-                     * Example:
-                     * EMPNAME, COMPANY
-                     * 'emp'aa','comab'
-                     * 'empbb','com'cd'
-                     * Here enclosure after emp(emp') and after com(com')
-                     * are not the last enclosures
-                     * </pre>
-                     */
-                    keepGoing = !(data.delimiterMatcher
-                        .matchesPattern(this.byteBuffer, this.endBuffer, data.delimiter)
-                        || data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer)
-                        || data.crLfMatcher.isLineFeed(this.byteBuffer, this.endBuffer));
-                  }
+              }
+            } while (keepGoing);
 
-                }
-              } while (keepGoing);
+            // Did we reach the end of the buffer?
+            //
+            if (this.endBuffer >= this.bufferSize) {
+              newLineFound = true; // consider it a newline to break out of the upper
+              // while loop
+              newLines += 2; // to remove the enclosures in case of missing
+              // newline on last line.
+              endOfBuffer = true;
+              break;
+            }
+          } else {
 
-              // Did we reach the end of the buffer?
-              //
+            this.endBuffer++;
+            this.currentOffset++;
+            this.totalBytesRead++;
+
+            if (checkBufferSize()) {
               if (this.endBuffer >= this.bufferSize) {
-                newLineFound = true; // consider it a newline to break out of the upper
-                // while loop
-                newLines += 2; // to remove the enclosures in case of missing
-                // newline on last line.
-                endOfBuffer = true;
+                newLineFound = true;
                 break;
               }
-            } else {
-
-              this.endBuffer++;
-              this.currentOffset++;
-              this.totalBytesRead++;
-
-              if (checkBufferSize()) {
-                if (this.endBuffer >= this.bufferSize) {
-                  newLineFound = true;
-                  break;
-                }
-              }
             }
           }
+        }
 
-          // If we're still here, we found a delimiter..
-          // Since the starting point never changed really, we just can grab range:
-          //
-          //    [startBuffer-endBuffer[
-          //
-          // This is the part we want.
-          // data.byteBuffer[data.startBuffer]
-          //
-          int length = calculateFieldLength(newLineFound, newLines, enclosureFound, endOfBuffer);
+        // If we're still here, we found a delimiter..
+        // Since the starting point never changed really, we just can grab range:
+        //
+        //    [startBuffer-endBuffer[
+        //
+        // This is the part we want.
+        // data.byteBuffer[data.startBuffer]
+        //
+        int length =
+            calculateFieldLength(newLineFound, newLines, enclosureFound, endOfBuffer);
 
-          byte[] field = new byte[length];
-          System.arraycopy(this.byteBuffer, this.startBuffer, field, 0, length);
+        byte[] field = new byte[length];
+        System.arraycopy(this.byteBuffer, this.startBuffer, field, 0, length);
 
-          // Did we have any escaped characters in there?
-          //
-          if (escapedEnclosureFound > 0) {
-            field = this.removeEscapedEnclosures(field, escapedEnclosureFound);
-          }
+        // Did we have any escaped characters in there?
+        //
+        if (escapedEnclosureFound > 0) {
+          field = this.removeEscapedEnclosures(field, escapedEnclosureFound);
+        }
 
-          if (doConversions) {
-            if (meta.isLazyConversionActive()) {
-              outputRowData[outputIndex++] = field;
-            } else {
-              // We're not lazy so we convert the data right here and now.
-              // The convert object uses binary storage as such we just have to ask
-              // the native type from it.
-              // That will do the actual conversion.
+        if (doConversions) {
+          if (meta.isLazyConversionActive()) {
+            outputRowData[outputIndex++] = field;
+          } else {
+            // We're not lazy so we convert the data right here and now.
+            // The convert object uses binary storage as such we just have to ask
+            // the native type from it.
+            // That will do the actual conversion.
+            //
+            ValueMetaInterface sourceValueMeta =
+                data.convertRowMeta.getValueMeta(outputIndex);
+            try {
+              // when found a blank line, outputRowData will be filled as
+              // Object array = ["@NU#LL$!BLANKLINE", null, null, ... ]
+              if (field.length == 0 && newLineFound && outputIndex == 0) {
+                outputRowData[outputIndex++] = CarbonCommonConstants.BLANK_LINE_FLAG;
+              } else {
+                outputRowData[outputIndex++] =
+                  sourceValueMeta.convertBinaryStringToNativeType(field);
+              }
+            } catch (KettleValueException e) {
+              // There was a conversion error,
               //
-              ValueMetaInterface sourceValueMeta = data.convertRowMeta.getValueMeta(outputIndex);
-              try {
-                // when found a blank line, outputRowData will be filled as
-                // Object array = ["@NU#LL$!BLANKLINE", null, null, ... ]
-                if (field.length == 0 && newLineFound && outputIndex == 0) {
-                  outputRowData[outputIndex++] = CarbonCommonConstants.BLANK_LINE_FLAG;
-                } else {
-                  outputRowData[outputIndex++] =
-                      sourceValueMeta.convertBinaryStringToNativeType(field);
-                }
-              } catch (KettleValueException e) {
-                // There was a conversion error,
-                //
-                outputRowData[outputIndex++] = null;
-
-                if (conversionExceptions == null) {
-                  conversionExceptions =
-                      new ArrayList<Exception>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-                  exceptionFields =
-                      new ArrayList<ValueMetaInterface>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-                }
+              outputRowData[outputIndex++] = null;
 
-                conversionExceptions.add(e);
-                exceptionFields.add(sourceValueMeta);
+              if (conversionExceptions == null) {
+                conversionExceptions = new ArrayList<Exception>(
+                    CarbonCommonConstants.CONSTANT_SIZE_TEN);
+                exceptionFields = new ArrayList<ValueMetaInterface>(
+                    CarbonCommonConstants.CONSTANT_SIZE_TEN);
               }
+
+              conversionExceptions.add(e);
+              exceptionFields.add(sourceValueMeta);
             }
-          } else {
-            outputRowData[outputIndex++] = null; // nothing for the header, no conversions here.
           }
+        } else {
+          outputRowData[outputIndex++] =
+              null; // nothing for the header, no conversions here.
+        }
 
-          // OK, move on to the next field...
-          if (!newLineFound) {
-            this.endBuffer++;
-            this.currentOffset++;
-            this.totalBytesRead++;
+        // OK, move on to the next field...
+        if (!newLineFound) {
+          this.endBuffer++;
+          this.currentOffset++;
+          this.totalBytesRead++;
+        }
+        this.startBuffer = this.endBuffer;
+      }
+
+      // See if we reached the end of the line.
+      // If not, we need to skip the remaining items on the line until the next newline...
+      if (!newLineFound && !checkBufferSize()) {
+        while (!data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer)
+                && !data.crLfMatcher.isLineFeed(this.byteBuffer, this.endBuffer)){
+          this.endBuffer++;
+          this.currentOffset++;
+          this.totalBytesRead++;
+
+          if (checkBufferSize()) {
+            break; // nothing more to read.
           }
-          this.startBuffer = this.endBuffer;
+
+          // HANDLE: if we're using quoting we might be dealing with a very dirty file
+          // with quoted newlines in trailing fields. (imagine that)
+          // In that particular case we want to use the same logic we use above
+          // (refactored a bit) to skip these fields.
+
         }
 
-        // See if we reached the end of the line.
-        // If not, we need to skip the remaining items on the line until the next newline...
-        if (!newLineFound && !checkBufferSize()) {
-          while (!data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer) && !data.crLfMatcher
-              .isLineFeed(this.byteBuffer, this.endBuffer)) {
+        if (!checkBufferSize()) {
+          while (data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer)
+              || data.crLfMatcher.isLineFeed(this.byteBuffer, this.endBuffer)) {
             this.endBuffer++;
             this.currentOffset++;
             this.totalBytesRead++;
-
             if (checkBufferSize()) {
               break; // nothing more to read.
             }
-
-            // HANDLE: if we're using quoting we might be dealing with a very dirty file
-            // with quoted newlines in trailing fields. (imagine that)
-            // In that particular case we want to use the same logic we use above
-            // (refactored a bit) to skip these fields.
-
           }
+        }
 
-          if (!checkBufferSize()) {
-            while (data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer) || data.crLfMatcher
-                .isLineFeed(this.byteBuffer, this.endBuffer)) {
-              this.endBuffer++;
-              this.currentOffset++;
-              this.totalBytesRead++;
-              if (checkBufferSize()) {
-                break; // nothing more to read.
-              }
-            }
-          }
+        // Make sure we start at the right position the next time around.
+        this.startBuffer = this.endBuffer;
+      }
 
-          // Make sure we start at the right position the next time around.
-          this.startBuffer = this.endBuffer;
-        }
 
-        //            incrementLinesInput();
-        if (conversionExceptions != null && conversionExceptions.size() > 0) {
-          // Forward the first exception
-          throw new KettleConversionException(
-              "There were " + conversionExceptions.size() + " conversion errors on line ",
-              conversionExceptions, exceptionFields, outputRowData);
-        }
-        if (outputIndex > 0 && outputIndex < meta.getInputFields().length) {
-          badRecordslogger.addBadRecordsToBilder(outputRowData, meta.getInputFields().length,
-              "Row record is not in valid csv format.", null);
-          continue;
-        } else {
-          return outputRowData;
-        }
+      //            incrementLinesInput();
+      if (conversionExceptions != null && conversionExceptions.size() > 0) {
+        // Forward the first exception
+        throw new KettleConversionException("There were " + conversionExceptions.size()
+            + " conversion errors on line ", conversionExceptions, exceptionFields, outputRowData);
       }
+
+      return outputRowData;
     } catch (KettleConversionException e) {
       throw e;
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6c2469db/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
index 525902d..76d5716 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
@@ -1,4 +1,4 @@
-/*
+/**
  * 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
@@ -386,7 +386,8 @@ public class CsvInput extends BaseStep implements StepInterface {
           doProcess();
           LOGGER.info("*****************Completed csv reading by thread***********");
         } catch (Throwable e) {
-          LOGGER.error(e, "Thread is terminated due to error");
+          LOGGER.error(e,
+              "Thread is terminated due to error");
         }
         return null;
       }
@@ -477,9 +478,6 @@ public class CsvInput extends BaseStep implements StepInterface {
       if (blockDataHandler.bufferedInputStream != null) {
         blockDataHandler.bufferedInputStream.close();
       }
-      if (null != blockDataHandler.badRecordslogger) {
-        blockDataHandler.badRecordslogger.closeStreams();
-      }
     } catch (RuntimeException e) {
       throw e;
     } catch (Exception e) {
@@ -588,4 +586,4 @@ public class CsvInput extends BaseStep implements StepInterface {
     return false;
   }
 
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6c2469db/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputMeta.java b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputMeta.java
index b448f02..6f895b1 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputMeta.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputMeta.java
@@ -49,7 +49,13 @@ import org.pentaho.di.resource.ResourceNamingInterface;
 import org.pentaho.di.resource.ResourceReference;
 import org.pentaho.di.trans.Trans;
 import org.pentaho.di.trans.TransMeta;
-import org.pentaho.di.trans.step.*;
+import org.pentaho.di.trans.step.BaseStepMeta;
+import org.pentaho.di.trans.step.StepDataInterface;
+import org.pentaho.di.trans.step.StepInjectionMetaEntry;
+import org.pentaho.di.trans.step.StepInterface;
+import org.pentaho.di.trans.step.StepMeta;
+import org.pentaho.di.trans.step.StepMetaInjectionInterface;
+import org.pentaho.di.trans.step.StepMetaInterface;
 import org.pentaho.di.trans.steps.textfileinput.InputFileMetaInterface;
 import org.pentaho.di.trans.steps.textfileinput.TextFileInputField;
 import org.pentaho.di.trans.steps.textfileinput.TextFileInputMeta;
@@ -90,14 +96,6 @@ public class CsvInputMeta extends BaseStepMeta
   private int currentRestructNumber;
 
   private String blocksID;
-  /**
-   * database name
-   */
-  private String databaseName;
-  /**
-   * tableName
-   */
-  private String tableName;
 
   public CsvInputMeta() {
     super(); // allocate BaseStepMeta
@@ -118,16 +116,10 @@ public class CsvInputMeta extends BaseStepMeta
     bufferSize = "50000";
     currentRestructNumber = -1;
     blocksID = "";
-    databaseName = "";
-    tableName = "";
-
-
   }
 
   private void readData(Node stepnode) throws KettleXMLException {
     try {
-      databaseName = XMLHandler.getTagValue(stepnode, "databaseName");
-      tableName = XMLHandler.getTagValue(stepnode, "tableName");
       filename = XMLHandler.getTagValue(stepnode, getXmlCode("FILENAME"));
       filenameField = XMLHandler.getTagValue(stepnode, getXmlCode("FILENAME_FIELD"));
       rowNumField = XMLHandler.getTagValue(stepnode, getXmlCode("ROW_NUM_FIELD"));
@@ -197,8 +189,7 @@ public class CsvInputMeta extends BaseStepMeta
 
   public String getXML() {
     StringBuffer retval = new StringBuffer(500);
-    retval.append("    ").append(XMLHandler.addTagValue("databaseName", databaseName));
-    retval.append("    ").append(XMLHandler.addTagValue("tableName", tableName));
+
     retval.append("    ").append(XMLHandler.addTagValue(getXmlCode("FILENAME"), filename));
     retval.append("    ")
         .append(XMLHandler.addTagValue(getXmlCode("FILENAME_FIELD"), filenameField));
@@ -257,8 +248,6 @@ public class CsvInputMeta extends BaseStepMeta
   public void readRep(Repository rep, ObjectId idStep, List<DatabaseMeta> databases,
       Map<String, Counter> counters) throws KettleException {
     try {
-      databaseName = rep.getStepAttributeString(idStep, getRepCode("databaseName"));
-      tableName = rep.getStepAttributeString(idStep, getRepCode("tableName"));
       filename = rep.getStepAttributeString(idStep, getRepCode("FILENAME"));
       filenameField = rep.getStepAttributeString(idStep, getRepCode("FILENAME_FIELD"));
       rowNumField = rep.getStepAttributeString(idStep, getRepCode("ROW_NUM_FIELD"));
@@ -309,8 +298,6 @@ public class CsvInputMeta extends BaseStepMeta
   public void saveRep(Repository rep, ObjectId idTransformation, ObjectId idStep)
       throws KettleException {
     try {
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("databaseName"), databaseName);
-      rep.saveStepAttribute(idTransformation, idStep, getRepCode("databaseName"), tableName);
       rep.saveStepAttribute(idTransformation, idStep, getRepCode("FILENAME"), filename);
       rep.saveStepAttribute(idTransformation, idStep, getRepCode("FILENAME_FIELD"), filenameField);
       rep.saveStepAttribute(idTransformation, idStep, getRepCode("ROW_NUM_FIELD"), rowNumField);
@@ -794,11 +781,7 @@ public class CsvInputMeta extends BaseStepMeta
       //
       String attributeKey = attr.getKey();
       if (entry.getValueType() != ValueMetaInterface.TYPE_NONE) {
-        if ("databaseName".equals(attributeKey)) {
-          databaseName = (String) entry.getValue();
-        } else if ("tableName".equals(attributeKey)) {
-          tableName = (String) entry.getValue();
-        } else if ("FILENAME".equals(attributeKey)) {
+        if ("FILENAME".equals(attributeKey)) {
           filename = (String) entry.getValue();
         } else if ("FILENAME_FIELD".equals(attributeKey)) {
           filenameField = (String) entry.getValue();
@@ -922,35 +905,4 @@ public class CsvInputMeta extends BaseStepMeta
     this.currentRestructNumber = currentRestructNum;
   }
 
-  /**
-   * retuns database name
-   * @return
-   */
-  public String getDatabaseName() {
-    return databaseName;
-  }
-
-  /**
-   * return tableName
-   * @return
-   */
-  public String getTableName() {
-    return tableName;
-  }
-
-  /**
-   * set databasename
-   * @param databaseName
-   */
-  public void setDatabaseName(String databaseName) {
-    this.databaseName = databaseName;
-  }
-
-  /**
-   * set tabke name
-   * @param tableName
-   */
-  public void setTableName(String tableName) {
-    this.tableName = tableName;
-  }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6c2469db/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java b/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
index 682b3d1..00f233c 100644
--- a/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
+++ b/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
@@ -440,8 +440,7 @@ public class GraphGenerator {
     csvInputMeta.setBlocksID(this.blocksID);
     csvDataStep.setDraw(true);
     csvDataStep.setDescription("Read raw data from " + GraphGeneratorConstants.CSV_INPUT);
-    csvInputMeta.setDatabaseName(schemaInfo.getSchemaName());
-    csvInputMeta.setTableName(schemaInfo.getCubeName());
+
     return csvDataStep;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6c2469db/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
index 6ac5722..e40bd25 100644
--- a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
@@ -27,8 +27,15 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.nio.charset.Charset;
 import java.sql.Connection;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Set;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -362,10 +369,9 @@ public class CarbonCSVBasedSeqGenStep extends BaseStep {
           columnsInfo.setDimensionColumnIds(meta.getDimensionColumnIds());
           columnsInfo.setColumnSchemaDetailsWrapper(meta.getColumnSchemaDetailsWrapper());
           updateBagLogFileName();
-          csvFilepath = CarbonDataProcessorUtil.getBagLogFileName(csvFilepath);
           String key = meta.getSchemaName() + '/' + meta.getCubeName() + '_' + meta.getTableName();
-          badRecordslogger = new BadRecordslogger(key, csvFilepath, CarbonDataProcessorUtil
-              .getBadLogStoreLocation(meta.getSchemaName() + '/' + meta.getCubeName()));
+          badRecordslogger = new BadRecordslogger(key, csvFilepath,
+              getBadLogStoreLocation(meta.getSchemaName() + '/' + meta.getCubeName()));
 
           columnsInfo.setTimeOrdinalIndices(meta.timeOrdinalIndices);
           surrogateKeyGen = new FileStoreSurrogateKeyGenForCSV(columnsInfo, meta.getPartitionID(),
@@ -694,6 +700,14 @@ public class CarbonCSVBasedSeqGenStep extends BaseStep {
             meta.getPartitionID(), meta.getSegmentId()+"");
   }
 
+  private String getBadLogStoreLocation(String storeLocation) {
+    String badLogStoreLocation =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
+    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
+
+    return badLogStoreLocation;
+  }
+
   private void updateBagLogFileName() {
     csvFilepath = new File(csvFilepath).getName();
     if (csvFilepath.indexOf(".") > -1) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6c2469db/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
index a64a256..8f2c9e7 100644
--- a/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -79,6 +79,7 @@ public final class CarbonDataProcessorUtil {
     return fileBufferSize;
   }
 
+
   /**
    * Utility method to get level cardinality string
    *
@@ -153,6 +154,7 @@ public final class CarbonDataProcessorUtil {
     }// CHECKSTYLE:ON
   }
 
+
   public static void checkResult(List<CheckResultInterface> remarks, StepMeta stepMeta,
       String[] input) {
     CheckResult cr;
@@ -274,34 +276,4 @@ public final class CarbonDataProcessorUtil {
     String localDataLoadFolderLocation = carbonDataDirectoryPath + File.separator + taskId;
     return localDataLoadFolderLocation;
   }
-
-  /**
-   * The method returns the bad record store location
-   *
-   * @param storeLocation
-   * @return
-   */
-  public static String getBadLogStoreLocation(String storeLocation) {
-    String badLogStoreLocation =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.CARBON_BADRECORDS_LOC);
-    badLogStoreLocation = badLogStoreLocation + File.separator + storeLocation;
-
-    return badLogStoreLocation;
-  }
-
-  /**
-   * method returns the bad log file name
-   *
-   * @param csvFilepath
-   * @return
-   */
-  public static String getBagLogFileName(String csvFilepath) {
-    csvFilepath = new File(csvFilepath).getName();
-    if (csvFilepath.indexOf(".") > -1) {
-      csvFilepath = csvFilepath.substring(0, csvFilepath.indexOf("."));
-    }
-
-    return csvFilepath + '_' + System.currentTimeMillis() + ".log";
-
-  }
-}
+}
\ No newline at end of file



[22/56] [abbrv] incubator-carbondata git commit: Optimized detail query flow and cleanup (#691)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
index 6ad4648..ce0f656 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
@@ -186,6 +186,7 @@ public class CarbonCompactionExecutor {
     model.setCountStarQuery(false);
     model.setDetailQuery(true);
     model.setForcedDetailRawQuery(true);
+    model.setRawBytesDetailQuery(true);
     model.setFilterExpressionResolverTree(null);
 
     List<QueryDimension> dims = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/PartitionImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/PartitionImpl.java b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/PartitionImpl.java
index 8abbc4c..6ea4c2d 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/PartitionImpl.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/PartitionImpl.java
@@ -28,12 +28,8 @@
  */
 package org.carbondata.spark.partition.api.impl;
 
-import java.util.HashMap;
 import java.util.List;
-import java.util.Map;
 
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevelFilter;
 import org.carbondata.spark.partition.api.Partition;
 
 public class PartitionImpl implements Partition {
@@ -41,9 +37,6 @@ public class PartitionImpl implements Partition {
   private String uniqueID;
   private String folderPath;
 
-  private Map<String, CarbonDimensionLevelFilter> filterMap =
-      new HashMap<String, CarbonDimensionLevelFilter>(
-          CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
   public PartitionImpl(String uniqueID, String folderPath) {
     this.uniqueID = uniqueID;
@@ -58,10 +51,6 @@ public class PartitionImpl implements Partition {
     return folderPath;
   }
 
-  public void setPartitionDetails(String columnName, CarbonDimensionLevelFilter filter) {
-    filterMap.put(columnName, filter);
-  }
-
   @Override public String toString() {
     return "{PartitionID -> " + uniqueID + " Path: " + folderPath + '}';
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
index 5f00874..cae2f28 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
@@ -22,20 +22,18 @@ package org.carbondata.spark.partition.api.impl;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.List;
-import java.util.Map;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.query.carbon.model.CarbonQueryPlan;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevelFilter;
 import org.carbondata.spark.partition.api.DataPartitioner;
 import org.carbondata.spark.partition.api.Partition;
 
 import org.apache.spark.sql.execution.command.Partitioner;
 
 /**
- * Sample partition based on MSISDN.
+ * Sample partition.
  */
 public class SampleDataPartitionerImpl implements DataPartitioner {
   private static final LogService LOGGER =
@@ -78,13 +76,9 @@ public class SampleDataPartitionerImpl implements DataPartitioner {
       PartitionImpl partitionImpl =
           new PartitionImpl("" + partionCounter, baseLocation + '/' + partionCounter);
 
-      CarbonDimensionLevelFilter filter = new CarbonDimensionLevelFilter();
       List<Object> includedHashes = new ArrayList<Object>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
       includedHashes.add(partionCounter);
 
-      filter.setIncludeFilter(includedHashes);
-      partitionImpl.setPartitionDetails(partitionColumn, filter);
-
       allPartitions.add(partitionImpl);
     }
   }
@@ -121,20 +115,9 @@ public class SampleDataPartitionerImpl implements DataPartitioner {
   /**
    * Identify the partitions applicable for the given filter
    */
-  public List<Partition> getPartitions(Map<String, CarbonDimensionLevelFilter> filters) {
-    if (filters == null || filters.size() == 0 || filters.get(partitionColumn) == null) {
-      return allPartitions;
-    }
-
-    CarbonDimensionLevelFilter msisdnFilter = filters.get(partitionColumn);
-    List<Partition> allowedPartitions =
-        new ArrayList<Partition>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+  public List<Partition> getPartitions() {
+    return allPartitions;
 
-    if (msisdnFilter.getIncludeFilter().isEmpty()) {
-      // Partition check can be done only for include filter list.
-      // If the filter is of other type,return all the partitions list
-      return allPartitions;
-    }
     // TODO: this has to be redone during partitioning implementation
     //    for (Partition aPartition : allPartitions) {
     //      CarbonDimensionLevelFilter partitionFilterDetails =
@@ -151,7 +134,6 @@ public class SampleDataPartitionerImpl implements DataPartitioner {
     //      }
     //    }
 
-    return allowedPartitions;
   }
 
   private int hashCode(long key) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 4c7c460..5359009 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -159,6 +159,7 @@ case class CarbonDictionaryDecoder(
               .createCache(CacheType.FORWARD_DICTIONARY, storePath)
           val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
             forwardDictionaryCache)
+          val dictIndex = dicts.zipWithIndex.filter(x => x._1 != null).map(x => x._2)
           new Iterator[InternalRow] {
             val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
             override final def hasNext: Boolean = iter.hasNext
@@ -166,15 +167,15 @@ case class CarbonDictionaryDecoder(
             override final def next(): InternalRow = {
               val row: InternalRow = iter.next()
               val data = row.toSeq(dataTypes).toArray
-              for (i <- data.indices) {
-                if (dicts(i) != null && data(i) != null) {
-                  data(i) = toType(DataTypeUtil
-                    .getDataBasedOnDataType(dicts(i)
-                      .getDictionaryValueForKey(data(i).asInstanceOf[Integer]),
-                      getDictionaryColumnIds(i)._3))
+              dictIndex.foreach { index =>
+                if (data(index) != null) {
+                  data(index) = DataTypeUtil
+                    .getDataBasedOnDataType(dicts(index)
+                      .getDictionaryValueForKey(data(index).asInstanceOf[Int]),
+                      getDictionaryColumnIds(index)._3)
                 }
               }
-              unsafeProjection(new GenericMutableRow(data))
+              unsafeProjection(row)
             }
           }
         }
@@ -191,13 +192,6 @@ case class CarbonDictionaryDecoder(
     }
   }
 
-  private def toType(obj: Any): Any = {
-    obj match {
-      case s: String => UTF8String.fromString(s)
-      case _ => obj
-    }
-  }
-
   private def getDictionary(atiMap: Map[String, AbsoluteTableIdentifier],
       cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
     val dicts: Seq[Dictionary] = getDictionaryColumnIds.map { f =>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index ba4c37e..22fa4fb 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -187,12 +187,6 @@ case class CarbonScan(
   override def outputsUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
 
   override def doExecute(): RDD[InternalRow] = {
-    def toType(obj: Any): Any = {
-      obj match {
-        case s: String => UTF8String.fromString(s)
-        case _ => obj
-      }
-    }
     val outUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
     inputRdd.mapPartitions { iter =>
       val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
@@ -201,9 +195,9 @@ case class CarbonScan(
 
         override def next(): InternalRow =
           if (outUnsafeRows) {
-            unsafeProjection(new GenericMutableRow(iter.next().map(toType)))
+            unsafeProjection(new GenericMutableRow(iter.next()))
           } else {
-            new GenericMutableRow(iter.next().map(toType))
+            new GenericMutableRow(iter.next())
           }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/processing/src/test/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator_UT.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator_UT.java b/processing/src/test/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator_UT.java
index 6ac8b60..601bae4 100644
--- a/processing/src/test/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator_UT.java
+++ b/processing/src/test/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator_UT.java
@@ -36,7 +36,7 @@ public class TimeStampDirectDictionaryGenerator_UT {
   private int surrogateKey = -1;
 
   @Before public void setUp() throws Exception {
-    TimeStampDirectDictionaryGenerator generator = new TimeStampDirectDictionaryGenerator();
+    TimeStampDirectDictionaryGenerator generator = TimeStampDirectDictionaryGenerator.instance;
     surrogateKey = generator.generateDirectSurrogateKey("2015-10-20 12:30:01");
   }
 
@@ -46,7 +46,7 @@ public class TimeStampDirectDictionaryGenerator_UT {
    * @throws Exception
    */
   @Test public void generateDirectSurrogateKey() throws Exception {
-    TimeStampDirectDictionaryGenerator generator = new TimeStampDirectDictionaryGenerator();
+    TimeStampDirectDictionaryGenerator generator = TimeStampDirectDictionaryGenerator.instance;
     // default timestamp format is "yyyy-MM-dd HH:mm:ss" and the data being passed
     // in "dd/MM/yyyy" so the parsing should fail and method should return -1.
     int key = generator.generateDirectSurrogateKey("20/12/2014");
@@ -62,7 +62,7 @@ public class TimeStampDirectDictionaryGenerator_UT {
    * @throws Exception
    */
   @Test public void getValueFromSurrogate() throws Exception {
-    TimeStampDirectDictionaryGenerator generator = new TimeStampDirectDictionaryGenerator();
+    TimeStampDirectDictionaryGenerator generator = TimeStampDirectDictionaryGenerator.instance;
     long valueFromSurrogate = (long) generator.getValueFromSurrogate(surrogateKey);
     Date date = new Date(valueFromSurrogate / 1000);
     SimpleDateFormat timeParser = new SimpleDateFormat(CarbonProperties.getInstance()


[29/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java b/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
new file mode 100644
index 0000000..802682b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
@@ -0,0 +1,1233 @@
+/*
+ * 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.carbondata.scan.filter;
+
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.Date;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.SortedMap;
+import java.util.TreeMap;
+import java.util.TreeSet;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.cache.Cache;
+import org.carbondata.core.cache.CacheProvider;
+import org.carbondata.core.cache.CacheType;
+import org.carbondata.core.cache.dictionary.Dictionary;
+import org.carbondata.core.cache.dictionary.DictionaryChunksWrapper;
+import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.carbondata.core.cache.dictionary.ForwardDictionary;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.IndexKey;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.core.keygenerator.KeyGenerator;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.core.util.CarbonProperties;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.core.util.CarbonUtilException;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.LiteralExpression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.executer.AndFilterExecuterImpl;
+import org.carbondata.scan.filter.executer.ColGroupFilterExecuterImpl;
+import org.carbondata.scan.filter.executer.DimColumnExecuterFilterInfo;
+import org.carbondata.scan.filter.executer.ExcludeFilterExecuterImpl;
+import org.carbondata.scan.filter.executer.FilterExecuter;
+import org.carbondata.scan.filter.executer.IncludeFilterExecuterImpl;
+import org.carbondata.scan.filter.executer.OrFilterExecuterImpl;
+import org.carbondata.scan.filter.executer.RestructureFilterExecuterImpl;
+import org.carbondata.scan.filter.executer.RowLevelFilterExecuterImpl;
+import org.carbondata.scan.filter.executer.RowLevelRangeTypeExecuterFacory;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.intf.RowImpl;
+import org.carbondata.scan.filter.intf.RowIntf;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+import org.carbondata.scan.filter.resolver.RowLevelFilterResolverImpl;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.util.DataTypeUtil;
+
+public final class FilterUtil {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(FilterUtil.class.getName());
+
+  private FilterUtil() {
+
+  }
+
+  /**
+   * Pattern used : Visitor Pattern
+   * Method will create filter executer tree based on the filter resolved tree,
+   * in this algorithm based on the resolver instance the executers will be visited
+   * and the resolved surrogates will be converted to keys
+   *
+   * @param filterExpressionResolverTree
+   * @param segmentProperties
+   * @return FilterExecuter instance
+   */
+  private static FilterExecuter createFilterExecuterTree(
+      FilterResolverIntf filterExpressionResolverTree, SegmentProperties segmentProperties) {
+    FilterExecuterType filterExecuterType = filterExpressionResolverTree.getFilterExecuterType();
+    switch (filterExecuterType) {
+      case INCLUDE:
+        return getIncludeFilterExecuter(filterExpressionResolverTree.getDimColResolvedFilterInfo(),
+            segmentProperties);
+      case EXCLUDE:
+        return new ExcludeFilterExecuterImpl(
+            filterExpressionResolverTree.getDimColResolvedFilterInfo(),
+            segmentProperties.getDimensionKeyGenerator());
+      case OR:
+        return new OrFilterExecuterImpl(
+            createFilterExecuterTree(filterExpressionResolverTree.getLeft(), segmentProperties),
+            createFilterExecuterTree(filterExpressionResolverTree.getRight(), segmentProperties));
+      case AND:
+        return new AndFilterExecuterImpl(
+            createFilterExecuterTree(filterExpressionResolverTree.getLeft(), segmentProperties),
+            createFilterExecuterTree(filterExpressionResolverTree.getRight(), segmentProperties));
+      case RESTRUCTURE:
+        return new RestructureFilterExecuterImpl(
+            filterExpressionResolverTree.getDimColResolvedFilterInfo(),
+            segmentProperties.getDimensionKeyGenerator());
+      case ROWLEVEL_LESSTHAN:
+      case ROWLEVEL_LESSTHAN_EQUALTO:
+      case ROWLEVEL_GREATERTHAN_EQUALTO:
+      case ROWLEVEL_GREATERTHAN:
+        return RowLevelRangeTypeExecuterFacory
+            .getRowLevelRangeTypeExecuter(filterExecuterType, filterExpressionResolverTree);
+      case ROWLEVEL:
+      default:
+        return new RowLevelFilterExecuterImpl(
+            ((RowLevelFilterResolverImpl) filterExpressionResolverTree)
+                .getDimColEvaluatorInfoList(),
+            ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getMsrColEvalutorInfoList(),
+            ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getFilterExpresion(),
+            ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier());
+
+    }
+
+  }
+
+  /**
+   * It gives filter executer based on columnar or column group
+   *
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   * @return
+   */
+  private static FilterExecuter getIncludeFilterExecuter(
+      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties) {
+
+    if (dimColResolvedFilterInfo.getDimension().isColumnar()) {
+      return new IncludeFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
+    } else {
+      return new ColGroupFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
+    }
+  }
+
+  /**
+   * This method will check if a given expression contains a column expression
+   * recursively.
+   *
+   * @return
+   */
+  public static boolean checkIfExpressionContainsColumn(Expression expression) {
+    if (expression instanceof ColumnExpression) {
+      return true;
+    }
+    for (Expression child : expression.getChildren()) {
+      if (checkIfExpressionContainsColumn(child)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * This method will check if a given expression contains a column expression
+   * recursively.
+   *
+   * @return
+   */
+  public static boolean checkIfExpressionContainsUnknownExp(Expression expression) {
+    if (expression.getFilterExpressionType() == ExpressionType.UNKNOWN) {
+      return true;
+    }
+    for (Expression child : expression.getChildren()) {
+      if (checkIfExpressionContainsUnknownExp(child)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * method will get the masked keys based on the keys generated from surrogates.
+   *
+   * @param ranges
+   * @param key
+   * @return byte[]
+   */
+  private static byte[] getMaskedKey(int[] ranges, byte[] key) {
+    byte[] maskkey = new byte[ranges.length];
+
+    for (int i = 0; i < maskkey.length; i++) {
+      maskkey[i] = key[ranges[i]];
+    }
+    return maskkey;
+  }
+
+  /**
+   * This method will return the ranges for the masked Bytes based on the key
+   * Generator.
+   *
+   * @param queryDimensionsOrdinal
+   * @param generator
+   * @return
+   */
+  private static int[] getRangesForMaskedByte(int queryDimensionsOrdinal, KeyGenerator generator) {
+    Set<Integer> integers = new TreeSet<Integer>();
+    int[] range = generator.getKeyByteOffsets(queryDimensionsOrdinal);
+    for (int j = range[0]; j <= range[1]; j++) {
+      integers.add(j);
+    }
+
+    int[] byteIndexs = new int[integers.size()];
+    int j = 0;
+    for (Iterator<Integer> iterator = integers.iterator(); iterator.hasNext(); ) {
+      Integer integer = iterator.next();
+      byteIndexs[j++] = integer.intValue();
+    }
+    return byteIndexs;
+  }
+
+  /**
+   * This method will get the no dictionary data based on filters and same
+   * will be in DimColumnFilterInfo
+   *
+   * @param tableIdentifier
+   * @param columnExpression
+   * @param evaluateResultListFinal
+   * @param isIncludeFilter
+   * @return DimColumnFilterInfo
+   */
+  public static DimColumnFilterInfo getNoDictionaryValKeyMemberForFilter(
+      AbsoluteTableIdentifier tableIdentifier, ColumnExpression columnExpression,
+      List<String> evaluateResultListFinal, boolean isIncludeFilter) {
+    List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
+    for (String result : evaluateResultListFinal) {
+      filterValuesList.add(result.getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)));
+    }
+
+    Comparator<byte[]> filterNoDictValueComaparator = new Comparator<byte[]>() {
+
+      @Override public int compare(byte[] filterMember1, byte[] filterMember2) {
+        // TODO Auto-generated method stub
+        return ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterMember1, filterMember2);
+      }
+
+    };
+    Collections.sort(filterValuesList, filterNoDictValueComaparator);
+    DimColumnFilterInfo columnFilterInfo = null;
+    if (filterValuesList.size() > 0) {
+      columnFilterInfo = new DimColumnFilterInfo();
+      columnFilterInfo.setIncludeFilter(isIncludeFilter);
+      columnFilterInfo.setFilterListForNoDictionaryCols(filterValuesList);
+
+    }
+    return columnFilterInfo;
+  }
+
+  /**
+   * Method will prepare the  dimfilterinfo instance by resolving the filter
+   * expression value to its respective surrogates.
+   *
+   * @param tableIdentifier
+   * @param columnExpression
+   * @param evaluateResultList
+   * @param isIncludeFilter
+   * @return
+   * @throws QueryExecutionException
+   */
+  public static DimColumnFilterInfo getFilterValues(AbsoluteTableIdentifier tableIdentifier,
+      ColumnExpression columnExpression, List<String> evaluateResultList, boolean isIncludeFilter)
+      throws QueryExecutionException, FilterUnsupportedException {
+    Dictionary forwardDictionary = null;
+    try {
+      // Reading the dictionary value from cache.
+      forwardDictionary =
+          getForwardDictionaryCache(tableIdentifier, columnExpression.getDimension());
+      return getFilterValues(columnExpression, evaluateResultList, forwardDictionary,
+          isIncludeFilter);
+    } finally {
+      CarbonUtil.clearDictionaryCache(forwardDictionary);
+    }
+  }
+
+  /**
+   * Method will prepare the  dimfilterinfo instance by resolving the filter
+   * expression value to its respective surrogates.
+   *
+   * @param columnExpression
+   * @param evaluateResultList
+   * @param forwardDictionary
+   * @param isIncludeFilter
+   * @return
+   * @throws QueryExecutionException
+   */
+  private static DimColumnFilterInfo getFilterValues(ColumnExpression columnExpression,
+      List<String> evaluateResultList, Dictionary forwardDictionary, boolean isIncludeFilter)
+      throws QueryExecutionException {
+    sortFilterModelMembers(columnExpression, evaluateResultList);
+    List<Integer> surrogates =
+        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    // Reading the dictionary value from cache.
+    getDictionaryValue(evaluateResultList, forwardDictionary, surrogates);
+    Collections.sort(surrogates);
+    DimColumnFilterInfo columnFilterInfo = null;
+    if (surrogates.size() > 0) {
+      columnFilterInfo = new DimColumnFilterInfo();
+      columnFilterInfo.setIncludeFilter(isIncludeFilter);
+      columnFilterInfo.setFilterList(surrogates);
+    }
+    return columnFilterInfo;
+  }
+
+  /**
+   * This API will get the Dictionary value for the respective filter member
+   * string.
+   *
+   * @param evaluateResultList filter value
+   * @param surrogates
+   * @throws QueryExecutionException
+   */
+  private static void getDictionaryValue(List<String> evaluateResultList,
+      Dictionary forwardDictionary, List<Integer> surrogates) throws QueryExecutionException {
+    ((ForwardDictionary) forwardDictionary)
+        .getSurrogateKeyByIncrementalSearch(evaluateResultList, surrogates);
+  }
+
+  /**
+   * This method will get all the members of column from the forward dictionary
+   * cache, this method will be basically used in row level filter resolver.
+   *
+   * @param tableIdentifier
+   * @param expression
+   * @param columnExpression
+   * @param isIncludeFilter
+   * @return DimColumnFilterInfo
+   * @throws QueryExecutionException
+   */
+  public static DimColumnFilterInfo getFilterListForAllValues(
+      AbsoluteTableIdentifier tableIdentifier, Expression expression,
+      final ColumnExpression columnExpression, boolean isIncludeFilter)
+      throws FilterUnsupportedException {
+    Dictionary forwardDictionary = null;
+    List<String> evaluateResultListFinal = new ArrayList<String>(20);
+    DictionaryChunksWrapper dictionaryWrapper = null;
+    try {
+      forwardDictionary =
+          getForwardDictionaryCache(tableIdentifier, columnExpression.getDimension());
+      dictionaryWrapper = forwardDictionary.getDictionaryChunks();
+      while (dictionaryWrapper.hasNext()) {
+        byte[] columnVal = dictionaryWrapper.next();
+        try {
+          RowIntf row = new RowImpl();
+          String stringValue =
+              new String(columnVal, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+          if (stringValue.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+            stringValue = null;
+          }
+          row.setValues(new Object[] { DataTypeUtil.getDataBasedOnDataType(stringValue,
+              columnExpression.getCarbonColumn().getDataType()) });
+          Boolean rslt = expression.evaluate(row).getBoolean();
+          if (null != rslt && !(rslt ^ isIncludeFilter)) {
+            if (null == stringValue) {
+              evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
+            } else {
+              evaluateResultListFinal.add(stringValue);
+            }
+          }
+        } catch (FilterUnsupportedException e) {
+          LOGGER.audit(e.getMessage());
+          throw new FilterUnsupportedException(e.getMessage());
+        }
+      }
+      return getFilterValues(columnExpression, evaluateResultListFinal, forwardDictionary,
+          isIncludeFilter);
+    } catch (QueryExecutionException e) {
+      throw new FilterUnsupportedException(e.getMessage());
+    } finally {
+      CarbonUtil.clearDictionaryCache(forwardDictionary);
+    }
+  }
+
+  private static void sortFilterModelMembers(final ColumnExpression columnExpression,
+      List<String> evaluateResultListFinal) {
+    Comparator<String> filterActualValueComaparator = new Comparator<String>() {
+
+      @Override public int compare(String filterMember1, String filterMember2) {
+        return compareFilterMembersBasedOnActualDataType(filterMember1, filterMember2,
+            columnExpression.getDataType());
+      }
+
+    };
+    Collections.sort(evaluateResultListFinal, filterActualValueComaparator);
+  }
+
+  /**
+   * Metahod will resolve the filter member to its respective surrogates by
+   * scanning the dictionary cache.
+   *
+   * @param tableIdentifier
+   * @param expression
+   * @param columnExpression
+   * @param isIncludeFilter
+   * @return
+   * @throws QueryExecutionException
+   */
+  public static DimColumnFilterInfo getFilterList(AbsoluteTableIdentifier tableIdentifier,
+      Expression expression, ColumnExpression columnExpression, boolean isIncludeFilter)
+      throws QueryExecutionException {
+    DimColumnFilterInfo resolvedFilterObject = null;
+    List<String> evaluateResultListFinal = new ArrayList<String>(20);
+    try {
+      List<ExpressionResult> evaluateResultList = expression.evaluate(null).getList();
+      for (ExpressionResult result : evaluateResultList) {
+        if (result.getString() == null) {
+          evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
+          continue;
+        }
+        evaluateResultListFinal.add(result.getString());
+      }
+
+      if (null != columnExpression.getCarbonColumn() && !columnExpression.getCarbonColumn()
+          .hasEncoding(Encoding.DICTIONARY)) {
+        resolvedFilterObject =
+            getNoDictionaryValKeyMemberForFilter(tableIdentifier, columnExpression,
+                evaluateResultListFinal, isIncludeFilter);
+      } else {
+        resolvedFilterObject =
+            getFilterValues(tableIdentifier, columnExpression, evaluateResultListFinal,
+                isIncludeFilter);
+      }
+    } catch (FilterUnsupportedException e) {
+      LOGGER.audit(e.getMessage());
+    }
+    return resolvedFilterObject;
+  }
+
+  /**
+   * Method will prepare the  dimfilterinfo instance by resolving the filter
+   * expression value to its respective surrogates in the scenario of restructure.
+   *
+   * @param expression
+   * @param columnExpression
+   * @param defaultValues
+   * @param defaultSurrogate
+   * @return
+   */
+  public static DimColumnFilterInfo getFilterListForRS(Expression expression,
+      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate) {
+    List<Integer> filterValuesList = new ArrayList<Integer>(20);
+    DimColumnFilterInfo columnFilterInfo = null;
+    // List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
+    List<String> evaluateResultListFinal = new ArrayList<String>(20);
+    // KeyGenerator keyGenerator =
+    // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate });
+    try {
+      List<ExpressionResult> evaluateResultList = expression.evaluate(null).getList();
+      for (ExpressionResult result : evaluateResultList) {
+        if (result.getString() == null) {
+          evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
+          continue;
+        }
+        evaluateResultListFinal.add(result.getString());
+      }
+
+      for (int i = 0; i < evaluateResultListFinal.size(); i++) {
+        if (evaluateResultListFinal.get(i).equals(defaultValues)) {
+          filterValuesList.add(defaultSurrogate);
+          break;
+        }
+      }
+      if (filterValuesList.size() > 0) {
+        columnFilterInfo = new DimColumnFilterInfo();
+        columnFilterInfo.setFilterList(filterValuesList);
+      }
+    } catch (FilterUnsupportedException e) {
+      LOGGER.audit(e.getMessage());
+    }
+    return columnFilterInfo;
+  }
+
+  /**
+   * This method will get the member based on filter expression evaluation from the
+   * forward dictionary cache, this method will be basically used in restructure.
+   *
+   * @param expression
+   * @param columnExpression
+   * @param defaultValues
+   * @param defaultSurrogate
+   * @param isIncludeFilter
+   * @return
+   */
+  public static DimColumnFilterInfo getFilterListForAllMembersRS(Expression expression,
+      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate,
+      boolean isIncludeFilter) {
+    List<Integer> filterValuesList = new ArrayList<Integer>(20);
+    List<String> evaluateResultListFinal = new ArrayList<String>(20);
+    DimColumnFilterInfo columnFilterInfo = null;
+
+    // KeyGenerator keyGenerator =
+    // KeyGeneratorFactory.getKeyGenerator(new int[] { defaultSurrogate });
+    try {
+      RowIntf row = new RowImpl();
+      if (defaultValues.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+        defaultValues = null;
+      }
+      row.setValues(new Object[] { DataTypeUtil.getDataBasedOnDataType(defaultValues,
+          columnExpression.getCarbonColumn().getDataType()) });
+      Boolean rslt = expression.evaluate(row).getBoolean();
+      if (null != rslt && !(rslt ^ isIncludeFilter)) {
+        if (null == defaultValues) {
+          evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
+        } else {
+          evaluateResultListFinal.add(defaultValues);
+        }
+      }
+    } catch (FilterUnsupportedException e) {
+      LOGGER.audit(e.getMessage());
+    }
+
+    if (null == defaultValues) {
+      defaultValues = CarbonCommonConstants.MEMBER_DEFAULT_VAL;
+    }
+    columnFilterInfo = new DimColumnFilterInfo();
+    for (int i = 0; i < evaluateResultListFinal.size(); i++) {
+      if (evaluateResultListFinal.get(i).equals(defaultValues)) {
+        filterValuesList.add(defaultSurrogate);
+        break;
+      }
+    }
+    columnFilterInfo.setFilterList(filterValuesList);
+    return columnFilterInfo;
+  }
+
+  public static byte[][] getKeyArray(DimColumnFilterInfo dimColumnFilterInfo,
+      CarbonDimension carbonDimension, KeyGenerator blockLevelKeyGenerator) {
+    if (!carbonDimension.hasEncoding(Encoding.DICTIONARY)) {
+      return dimColumnFilterInfo.getNoDictionaryFilterValuesList()
+          .toArray((new byte[dimColumnFilterInfo.getNoDictionaryFilterValuesList().size()][]));
+    }
+    int[] keys = new int[blockLevelKeyGenerator.getDimCount()];
+    List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
+    Arrays.fill(keys, 0);
+    int[] rangesForMaskedByte =
+        getRangesForMaskedByte((carbonDimension.getKeyOrdinal()), blockLevelKeyGenerator);
+    if (null != dimColumnFilterInfo) {
+      for (Integer surrogate : dimColumnFilterInfo.getFilterList()) {
+        try {
+          keys[carbonDimension.getKeyOrdinal()] = surrogate;
+          filterValuesList
+              .add(getMaskedKey(rangesForMaskedByte, blockLevelKeyGenerator.generateKey(keys)));
+        } catch (KeyGenException e) {
+          LOGGER.error(e.getMessage());
+        }
+      }
+    }
+    return filterValuesList.toArray(new byte[filterValuesList.size()][]);
+
+  }
+
+  /**
+   * Method will return the start key based on KeyGenerator for the respective
+   * filter resolved instance.
+   *
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   * @return long[] start key
+   */
+  public static long[] getStartKey(DimColumnResolvedFilterInfo dimColResolvedFilterInfo,
+      SegmentProperties segmentProperties, long[] startKey) {
+    Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter =
+        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
+    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
+      List<DimColumnFilterInfo> values = entry.getValue();
+      if (null == values || !entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
+        continue;
+      }
+      boolean isExcludePresent = false;
+      for (DimColumnFilterInfo info : values) {
+        if (!info.isIncludeFilter()) {
+          isExcludePresent = true;
+        }
+      }
+      if (isExcludePresent) {
+        continue;
+      }
+      getStartKeyBasedOnFilterResoverInfo(dimensionFilter, startKey);
+    }
+    return startKey;
+  }
+
+  /**
+   * Algorithm for getting the start key for a filter
+   * step 1: Iterate through each dimension and verify whether its not an exclude filter.
+   * step 2: Intialize start key with the first filter member value present in each filter model
+   * for the respective dimensions.
+   * step 3: since its a no dictionary start key there will only actual value so compare
+   * the first filter model value with respect to the dimension data type.
+   * step 4: The least value will be considered as the start key of dimension by comparing all
+   * its filter model.
+   * step 5: create a byte array of start key which comprises of least filter member value of
+   * all dimension and the indexes which will help to read the respective filter value.
+   *
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   * @param setOfStartKeyByteArray
+   * @return
+   */
+  public static void getStartKeyForNoDictionaryDimension(
+      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties,
+      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
+    Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter =
+        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
+    // step 1
+    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
+      if (!entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
+        List<DimColumnFilterInfo> listOfDimColFilterInfo = entry.getValue();
+        if (null == listOfDimColFilterInfo) {
+          continue;
+        }
+        boolean isExcludePresent = false;
+        for (DimColumnFilterInfo info : listOfDimColFilterInfo) {
+          if (!info.isIncludeFilter()) {
+            isExcludePresent = true;
+          }
+        }
+        if (isExcludePresent) {
+          continue;
+        }
+        // step 2
+        byte[] noDictionaryStartKey =
+            listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList().get(0);
+        if (setOfStartKeyByteArray.isEmpty()) {
+          setOfStartKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryStartKey);
+        } else if (null == setOfStartKeyByteArray.get(entry.getKey().getOrdinal())) {
+          setOfStartKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryStartKey);
+
+        } else if (ByteUtil.UnsafeComparer.INSTANCE
+            .compareTo(setOfStartKeyByteArray.get(entry.getKey().getOrdinal()),
+                noDictionaryStartKey) > 0) {
+          setOfStartKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryStartKey);
+        }
+      }
+    }
+  }
+
+  /**
+   * Algorithm for getting the end key for a filter
+   * step 1: Iterate through each dimension and verify whether its not an exclude filter.
+   * step 2: Initialize end key with the last filter member value present in each filter model
+   * for the respective dimensions.(Already filter models are sorted)
+   * step 3: since its a no dictionary end key there will only actual value so compare
+   * the last filter model value with respect to the dimension data type.
+   * step 4: The highest value will be considered as the end key of dimension by comparing all
+   * its filter model.
+   * step 5: create a byte array of end key which comprises of highest filter member value of
+   * all dimension and the indexes which will help to read the respective filter value.
+   *
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   * @param setOfEndKeyByteArray
+   * @return end key array
+   */
+  public static void getEndKeyForNoDictionaryDimension(
+      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties,
+      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
+
+    Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter =
+        dimColResolvedFilterInfo.getDimensionResolvedFilterInstance();
+    // step 1
+    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
+      if (!entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
+        List<DimColumnFilterInfo> listOfDimColFilterInfo = entry.getValue();
+        if (null == listOfDimColFilterInfo) {
+          continue;
+        }
+        boolean isExcludePresent = false;
+        for (DimColumnFilterInfo info : listOfDimColFilterInfo) {
+          if (!info.isIncludeFilter()) {
+            isExcludePresent = true;
+          }
+        }
+        if (isExcludePresent) {
+          continue;
+        }
+        // step 2
+        byte[] noDictionaryEndKey = listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList()
+            .get(listOfDimColFilterInfo.get(0).getNoDictionaryFilterValuesList().size() - 1);
+        if (setOfEndKeyByteArray.isEmpty()) {
+          setOfEndKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryEndKey);
+        } else if (null == setOfEndKeyByteArray.get(entry.getKey().getOrdinal())) {
+          setOfEndKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryEndKey);
+
+        } else if (ByteUtil.UnsafeComparer.INSTANCE
+            .compareTo(setOfEndKeyByteArray.get(entry.getKey().getOrdinal()), noDictionaryEndKey)
+            < 0) {
+          setOfEndKeyByteArray.put(entry.getKey().getOrdinal(), noDictionaryEndKey);
+        }
+
+      }
+    }
+  }
+
+  /**
+   * Method will pack all the byte[] to a single byte[] value by appending the
+   * indexes of the byte[] value which needs to be read. this method will be mailny used
+   * in case of no dictionary dimension processing for filters.
+   *
+   * @param noDictionaryValKeyList
+   * @return packed key with its indexes added in starting and its actual values.
+   */
+  private static byte[] getKeyWithIndexesAndValues(List<byte[]> noDictionaryValKeyList) {
+    ByteBuffer[] buffArr = new ByteBuffer[noDictionaryValKeyList.size()];
+    int index = 0;
+    for (byte[] singleColVal : noDictionaryValKeyList) {
+      buffArr[index] = ByteBuffer.allocate(singleColVal.length);
+      buffArr[index].put(singleColVal);
+      buffArr[index++].rewind();
+    }
+    // byteBufer.
+    return CarbonUtil.packByteBufferIntoSingleByteArray(buffArr);
+
+  }
+
+  /**
+   * This method will fill the start key array  with the surrogate key present
+   * in filterinfo instance.
+   *
+   * @param dimensionFilter
+   * @param startKey
+   */
+  private static void getStartKeyBasedOnFilterResoverInfo(
+      Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter, long[] startKey) {
+    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
+      List<DimColumnFilterInfo> values = entry.getValue();
+      if (null == values) {
+        continue;
+      }
+      boolean isExcludePresent = false;
+      for (DimColumnFilterInfo info : values) {
+        if (!info.isIncludeFilter()) {
+          isExcludePresent = true;
+        }
+      }
+      if (isExcludePresent) {
+        continue;
+      }
+      for (DimColumnFilterInfo info : values) {
+        if (startKey[entry.getKey().getKeyOrdinal()] < info.getFilterList().get(0)) {
+          startKey[entry.getKey().getKeyOrdinal()] = info.getFilterList().get(0);
+        }
+      }
+    }
+  }
+
+  public static void getEndKey(Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter,
+      AbsoluteTableIdentifier tableIdentifier, long[] endKey, SegmentProperties segmentProperties)
+      throws QueryExecutionException {
+
+    List<CarbonDimension> updatedDimListBasedOnKeyGenerator =
+        getCarbonDimsMappedToKeyGenerator(segmentProperties.getDimensions());
+    for (int i = 0; i < endKey.length; i++) {
+      endKey[i] = getMaxValue(tableIdentifier, updatedDimListBasedOnKeyGenerator.get(i),
+          segmentProperties.getDimColumnsCardinality());
+    }
+    getEndKeyWithFilter(dimensionFilter, endKey);
+
+  }
+
+  private static List<CarbonDimension> getCarbonDimsMappedToKeyGenerator(
+      List<CarbonDimension> carbonDimensions) {
+    List<CarbonDimension> listOfCarbonDimPartOfKeyGen =
+        new ArrayList<CarbonDimension>(carbonDimensions.size());
+    for (CarbonDimension carbonDim : carbonDimensions) {
+      if (CarbonUtil.hasEncoding(carbonDim.getEncoder(), Encoding.DICTIONARY) || CarbonUtil
+          .hasEncoding(carbonDim.getEncoder(), Encoding.DIRECT_DICTIONARY)) {
+        listOfCarbonDimPartOfKeyGen.add(carbonDim);
+      }
+
+    }
+    return listOfCarbonDimPartOfKeyGen;
+  }
+
+  private static void getEndKeyWithFilter(
+      Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionFilter, long[] endKey) {
+    for (Entry<CarbonDimension, List<DimColumnFilterInfo>> entry : dimensionFilter.entrySet()) {
+      List<DimColumnFilterInfo> values = entry.getValue();
+      if (null == values || !entry.getKey().hasEncoding(Encoding.DICTIONARY)) {
+        continue;
+      }
+      boolean isExcludeFilterPresent = false;
+      for (DimColumnFilterInfo info : values) {
+        if (!info.isIncludeFilter()) {
+          isExcludeFilterPresent = true;
+        }
+      }
+      if (isExcludeFilterPresent) {
+        continue;
+      }
+
+      for (DimColumnFilterInfo info : values) {
+        if (endKey[entry.getKey().getKeyOrdinal()] > info.getFilterList()
+            .get(info.getFilterList().size() - 1)) {
+          endKey[entry.getKey().getKeyOrdinal()] =
+              info.getFilterList().get(info.getFilterList().size() - 1);
+        }
+      }
+    }
+
+  }
+
+  /**
+   * This API will get the max value of surrogate key which will be used for
+   * determining the end key of particular btree.
+   *
+   * @param dimCarinality
+   * @throws QueryExecutionException
+   */
+  private static long getMaxValue(AbsoluteTableIdentifier tableIdentifier,
+      CarbonDimension carbonDimension, int[] dimCarinality) throws QueryExecutionException {
+    //    if (DataType.TIMESTAMP == carbonDimension.getDataType()) {
+    //      return Integer.MAX_VALUE;
+    //    }
+    // Get data from all the available slices of the cube
+    if (null != dimCarinality) {
+      return dimCarinality[carbonDimension.getKeyOrdinal()];
+    }
+    return -1;
+  }
+
+  /**
+   * @param tableIdentifier
+   * @param carbonDimension
+   * @return
+   * @throws QueryExecutionException
+   */
+  public static Dictionary getForwardDictionaryCache(AbsoluteTableIdentifier tableIdentifier,
+      CarbonDimension carbonDimension) throws QueryExecutionException {
+    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+        new DictionaryColumnUniqueIdentifier(tableIdentifier.getCarbonTableIdentifier(),
+            String.valueOf(carbonDimension.getColumnId()), carbonDimension.getDataType());
+    CacheProvider cacheProvider = CacheProvider.getInstance();
+    Cache forwardDictionaryCache =
+        cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, tableIdentifier.getStorePath());
+    // get the forward dictionary object
+    Dictionary forwardDictionary = null;
+    try {
+      forwardDictionary = (Dictionary) forwardDictionaryCache.get(dictionaryColumnUniqueIdentifier);
+    } catch (CarbonUtilException e) {
+      throw new QueryExecutionException(e);
+    }
+    return forwardDictionary;
+  }
+
+  public static IndexKey createIndexKeyFromResolvedFilterVal(long[] startOrEndKey,
+      KeyGenerator keyGenerator, byte[] startOrEndKeyForNoDictDimension) {
+    IndexKey indexKey = null;
+    try {
+      indexKey =
+          new IndexKey(keyGenerator.generateKey(startOrEndKey), startOrEndKeyForNoDictDimension);
+    } catch (KeyGenException e) {
+      LOGGER.error(e.getMessage());
+    }
+    return indexKey;
+  }
+
+  /**
+   * API will create an filter executer tree based on the filter resolver
+   *
+   * @param filterExpressionResolverTree
+   * @param segmentProperties
+   * @return
+   */
+  public static FilterExecuter getFilterExecuterTree(
+      FilterResolverIntf filterExpressionResolverTree, SegmentProperties segmentProperties) {
+    return createFilterExecuterTree(filterExpressionResolverTree, segmentProperties);
+  }
+
+  /**
+   * API will prepare the Keys from the surrogates of particular filter resolver
+   *
+   * @param filterValues
+   * @param blockKeyGenerator
+   * @param dimension
+   * @param dimColumnExecuterInfo
+   */
+  public static void prepareKeysFromSurrogates(DimColumnFilterInfo filterValues,
+      KeyGenerator blockKeyGenerator, CarbonDimension dimension,
+      DimColumnExecuterFilterInfo dimColumnExecuterInfo) {
+    byte[][] keysBasedOnFilter = getKeyArray(filterValues, dimension, blockKeyGenerator);
+    dimColumnExecuterInfo.setFilterKeys(keysBasedOnFilter);
+
+  }
+
+  /**
+   * method will create a default end key in case of no end key is been derived using existing
+   * filter or in case of non filter queries.
+   *
+   * @param segmentProperties
+   * @return
+   * @throws KeyGenException
+   */
+  public static IndexKey prepareDefaultEndIndexKey(SegmentProperties segmentProperties)
+      throws KeyGenException {
+    long[] dictionarySurrogateKey =
+        new long[segmentProperties.getDimensions().size() - segmentProperties
+            .getNumberOfNoDictionaryDimension()];
+    Arrays.fill(dictionarySurrogateKey, Long.MAX_VALUE);
+    IndexKey endIndexKey;
+    byte[] dictionaryendMdkey =
+        segmentProperties.getDimensionKeyGenerator().generateKey(dictionarySurrogateKey);
+    byte[] noDictionaryEndKeyBuffer = getNoDictionaryDefaultEndKey(segmentProperties);
+    endIndexKey = new IndexKey(dictionaryendMdkey, noDictionaryEndKeyBuffer);
+    return endIndexKey;
+  }
+
+  public static byte[] getNoDictionaryDefaultEndKey(SegmentProperties segmentProperties) {
+    // in case of non filter query when no dictionary columns are present we
+    // need to set the default end key, as for non filter query
+    // we need to get the last
+    // block of the btree so we are setting the max byte value in the end key
+    ByteBuffer noDictionaryEndKeyBuffer = ByteBuffer.allocate(
+        (segmentProperties.getNumberOfNoDictionaryDimension()
+            * CarbonCommonConstants.SHORT_SIZE_IN_BYTE) + segmentProperties
+            .getNumberOfNoDictionaryDimension());
+    // end key structure will be
+    //<Offset of first No Dictionary key in 2 Bytes><Offset of second No Dictionary key in 2 Bytes>
+    //<Offset of n No Dictionary key in 2 Bytes><first no dictionary column value>
+    // <second no dictionary column value> <N no dictionary column value>
+    //example if we have 2 no dictionary column
+    //<[0,4,0,5,127,127]>
+    short startPoint = (short) (segmentProperties.getNumberOfNoDictionaryDimension()
+        * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
+    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
+      noDictionaryEndKeyBuffer.putShort((startPoint));
+      startPoint++;
+    }
+    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
+      noDictionaryEndKeyBuffer.put((byte) 127);
+    }
+    return noDictionaryEndKeyBuffer.array();
+  }
+
+  /**
+   * method will create a default end key in case of no end key is been
+   * derived using existing filter or in case of non filter queries.
+   *
+   * @param segmentProperties
+   * @return
+   * @throws KeyGenException
+   */
+  public static IndexKey prepareDefaultStartIndexKey(SegmentProperties segmentProperties)
+      throws KeyGenException {
+    IndexKey startIndexKey;
+    long[] dictionarySurrogateKey =
+        new long[segmentProperties.getDimensions().size() - segmentProperties
+            .getNumberOfNoDictionaryDimension()];
+    byte[] dictionaryStartMdkey =
+        segmentProperties.getDimensionKeyGenerator().generateKey(dictionarySurrogateKey);
+    byte[] noDictionaryStartKeyArray = getNoDictionaryDefaultStartKey(segmentProperties);
+
+    startIndexKey = new IndexKey(dictionaryStartMdkey, noDictionaryStartKeyArray);
+    return startIndexKey;
+  }
+
+  public static byte[] getNoDictionaryDefaultStartKey(SegmentProperties segmentProperties) {
+    // in case of non filter query when no dictionary columns are present we
+    // need to set the default start key, as for non filter query we need to get the first
+    // block of the btree so we are setting the least byte value in the start key
+    ByteBuffer noDictionaryStartKeyBuffer = ByteBuffer.allocate(
+        (segmentProperties.getNumberOfNoDictionaryDimension()
+            * CarbonCommonConstants.SHORT_SIZE_IN_BYTE) + segmentProperties
+            .getNumberOfNoDictionaryDimension());
+    // end key structure will be
+    //<Offset of first No Dictionary key in 2 Bytes><Offset of second No Dictionary key in 2 Bytes>
+    //<Offset of n No Dictionary key in 2 Bytes><first no dictionary column value>
+    // <second no dictionary column value> <N no dictionary column value>
+    //example if we have 2 no dictionary column
+    //<[0,4,0,5,0,0]>
+    short startPoint = (short) (segmentProperties.getNumberOfNoDictionaryDimension()
+        * CarbonCommonConstants.SHORT_SIZE_IN_BYTE);
+    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
+      noDictionaryStartKeyBuffer.putShort((startPoint));
+      startPoint++;
+    }
+    for (int i = 0; i < segmentProperties.getNumberOfNoDictionaryDimension(); i++) {
+      noDictionaryStartKeyBuffer.put((byte) 0);
+    }
+    return noDictionaryStartKeyBuffer.array();
+  }
+
+  public static int compareFilterKeyBasedOnDataType(String dictionaryVal, String memberVal,
+      DataType dataType) {
+    try {
+      switch (dataType) {
+        case INT:
+
+          return Integer.compare((Integer.parseInt(dictionaryVal)), (Integer.parseInt(memberVal)));
+        case DOUBLE:
+          return Double
+              .compare((Double.parseDouble(dictionaryVal)), (Double.parseDouble(memberVal)));
+        case LONG:
+          return Long.compare((Long.parseLong(dictionaryVal)), (Long.parseLong(memberVal)));
+        case BOOLEAN:
+          return Boolean
+              .compare((Boolean.parseBoolean(dictionaryVal)), (Boolean.parseBoolean(memberVal)));
+        case TIMESTAMP:
+          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+          Date dateToStr;
+          Date dictionaryDate;
+          dateToStr = parser.parse(memberVal);
+          dictionaryDate = parser.parse(dictionaryVal);
+          return dictionaryDate.compareTo(dateToStr);
+
+        case DECIMAL:
+          java.math.BigDecimal javaDecValForDictVal = new java.math.BigDecimal(dictionaryVal);
+          java.math.BigDecimal javaDecValForMemberVal = new java.math.BigDecimal(memberVal);
+          return javaDecValForDictVal.compareTo(javaDecValForMemberVal);
+        default:
+          return -1;
+      }
+    } catch (Exception e) {
+      return -1;
+    }
+  }
+
+  /**
+   * method will set the start and end key for as per the filter resolver tree
+   * utilized visitor pattern inorder to populate the start and end key population.
+   *
+   * @param segmentProperties
+   * @param tableIdentifier
+   * @param filterResolver
+   * @param listOfStartEndKeys
+   * @throws QueryExecutionException
+   */
+  public static void traverseResolverTreeAndGetStartAndEndKey(SegmentProperties segmentProperties,
+      AbsoluteTableIdentifier tableIdentifier, FilterResolverIntf filterResolver,
+      List<IndexKey> listOfStartEndKeys) throws QueryExecutionException {
+    IndexKey searchStartKey = null;
+    IndexKey searchEndKey = null;
+    long[] startKey = new long[segmentProperties.getDimensionKeyGenerator().getDimCount()];
+    long[] endKey = new long[segmentProperties.getDimensionKeyGenerator().getDimCount()];
+    List<byte[]> listOfStartKeyByteArray =
+        new ArrayList<byte[]>(segmentProperties.getNumberOfNoDictionaryDimension());
+    List<byte[]> listOfEndKeyByteArray =
+        new ArrayList<byte[]>(segmentProperties.getNumberOfNoDictionaryDimension());
+    SortedMap<Integer, byte[]> setOfStartKeyByteArray = new TreeMap<Integer, byte[]>();
+    SortedMap<Integer, byte[]> setOfEndKeyByteArray = new TreeMap<Integer, byte[]>();
+    SortedMap<Integer, byte[]> defaultStartValues = new TreeMap<Integer, byte[]>();
+    SortedMap<Integer, byte[]> defaultEndValues = new TreeMap<Integer, byte[]>();
+    traverseResolverTreeAndPopulateStartAndEndKeys(filterResolver, tableIdentifier,
+        segmentProperties, startKey, setOfStartKeyByteArray, endKey, setOfEndKeyByteArray);
+    fillDefaultStartValue(defaultStartValues, segmentProperties);
+    fillDefaultEndValue(defaultEndValues, segmentProperties);
+    fillNullValuesStartIndexWithDefaultKeys(setOfStartKeyByteArray, segmentProperties);
+    fillNullValuesEndIndexWithDefaultKeys(setOfEndKeyByteArray, segmentProperties);
+    pruneStartAndEndKeys(setOfStartKeyByteArray, listOfStartKeyByteArray);
+    pruneStartAndEndKeys(setOfEndKeyByteArray, listOfEndKeyByteArray);
+
+    searchStartKey = FilterUtil
+        .createIndexKeyFromResolvedFilterVal(startKey, segmentProperties.getDimensionKeyGenerator(),
+            FilterUtil.getKeyWithIndexesAndValues(listOfStartKeyByteArray));
+
+    searchEndKey = FilterUtil
+        .createIndexKeyFromResolvedFilterVal(endKey, segmentProperties.getDimensionKeyGenerator(),
+            FilterUtil.getKeyWithIndexesAndValues(listOfEndKeyByteArray));
+    listOfStartEndKeys.add(searchStartKey);
+    listOfStartEndKeys.add(searchEndKey);
+
+  }
+
+  private static int compareFilterMembersBasedOnActualDataType(String filterMember1,
+      String filterMember2, org.carbondata.scan.expression.DataType dataType) {
+    try {
+      switch (dataType) {
+        case IntegerType:
+        case LongType:
+        case DoubleType:
+
+          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
+            return 1;
+          }
+          Double d1 = Double.parseDouble(filterMember1);
+          Double d2 = Double.parseDouble(filterMember2);
+          return d1.compareTo(d2);
+        case DecimalType:
+          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
+            return 1;
+          }
+          java.math.BigDecimal val1 = new BigDecimal(filterMember1);
+          java.math.BigDecimal val2 = new BigDecimal(filterMember2);
+          return val1.compareTo(val2);
+        case TimestampType:
+          if (CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(filterMember1)) {
+            return 1;
+          }
+          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+          Date date1 = null;
+          Date date2 = null;
+          date1 = parser.parse(filterMember1);
+          date2 = parser.parse(filterMember2);
+          return date1.compareTo(date2);
+        case StringType:
+        default:
+          return filterMember1.compareTo(filterMember2);
+      }
+    } catch (Exception e) {
+      return -1;
+    }
+  }
+
+  private static void fillNullValuesStartIndexWithDefaultKeys(
+      SortedMap<Integer, byte[]> setOfStartKeyByteArray, SegmentProperties segmentProperties) {
+    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
+    for (CarbonDimension dimension : allDimension) {
+      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
+        continue;
+      }
+      if (null == setOfStartKeyByteArray.get(dimension.getOrdinal())) {
+        setOfStartKeyByteArray.put(dimension.getOrdinal(), new byte[] { 0 });
+      }
+
+    }
+  }
+
+  private static void fillNullValuesEndIndexWithDefaultKeys(
+      SortedMap<Integer, byte[]> setOfStartKeyByteArray, SegmentProperties segmentProperties) {
+    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
+    for (CarbonDimension dimension : allDimension) {
+      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
+        continue;
+      }
+      if (null == setOfStartKeyByteArray.get(dimension.getOrdinal())) {
+        setOfStartKeyByteArray.put(dimension.getOrdinal(), new byte[] { 127 });
+      }
+
+    }
+  }
+
+  private static void pruneStartAndEndKeys(SortedMap<Integer, byte[]> setOfStartKeyByteArray,
+      List<byte[]> listOfStartKeyByteArray) {
+    for (Map.Entry<Integer, byte[]> entry : setOfStartKeyByteArray.entrySet()) {
+      listOfStartKeyByteArray.add(entry.getValue());
+    }
+  }
+
+  private static void fillDefaultStartValue(SortedMap<Integer, byte[]> setOfStartKeyByteArray,
+      SegmentProperties segmentProperties) {
+    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
+    for (CarbonDimension dimension : allDimension) {
+      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
+        continue;
+      }
+      setOfStartKeyByteArray.put(dimension.getOrdinal(), new byte[] { 0 });
+    }
+
+  }
+
+  private static void fillDefaultEndValue(SortedMap<Integer, byte[]> setOfEndKeyByteArray,
+      SegmentProperties segmentProperties) {
+    List<CarbonDimension> allDimension = segmentProperties.getDimensions();
+    for (CarbonDimension dimension : allDimension) {
+      if (CarbonUtil.hasEncoding(dimension.getEncoder(), Encoding.DICTIONARY)) {
+        continue;
+      }
+      setOfEndKeyByteArray.put(dimension.getOrdinal(), new byte[] { 127 });
+    }
+  }
+
+  private static void traverseResolverTreeAndPopulateStartAndEndKeys(
+      FilterResolverIntf filterResolverTree, AbsoluteTableIdentifier tableIdentifier,
+      SegmentProperties segmentProperties, long[] startKeys,
+      SortedMap<Integer, byte[]> setOfStartKeyByteArray, long[] endKeys,
+      SortedMap<Integer, byte[]> setOfEndKeyByteArray) throws QueryExecutionException {
+    if (null == filterResolverTree) {
+      return;
+    }
+    traverseResolverTreeAndPopulateStartAndEndKeys(filterResolverTree.getLeft(), tableIdentifier,
+        segmentProperties, startKeys, setOfStartKeyByteArray, endKeys, setOfEndKeyByteArray);
+
+    filterResolverTree.getStartKey(segmentProperties, startKeys, setOfStartKeyByteArray);
+    filterResolverTree.getEndKey(segmentProperties, tableIdentifier, endKeys, setOfEndKeyByteArray);
+
+    traverseResolverTreeAndPopulateStartAndEndKeys(filterResolverTree.getRight(), tableIdentifier,
+        segmentProperties, startKeys, setOfStartKeyByteArray, endKeys, setOfEndKeyByteArray);
+  }
+
+  /**
+   * Method will find whether the expression needs to be resolved, this can happen
+   * if the expression is exclude and data type is null(mainly in IS NOT NULL filter scenario)
+   * @param rightExp
+   * @param isIncludeFilter
+   * @return
+   */
+  public static boolean isExpressionNeedsToResolved(Expression rightExp, boolean isIncludeFilter) {
+    if (!isIncludeFilter && rightExp instanceof LiteralExpression && (
+        org.carbondata.scan.expression.DataType.NullType == ((LiteralExpression) rightExp)
+            .getLiteralExpDataType())) {
+      return true;
+    }
+    for (Expression child : rightExp.getChildren()) {
+      if (isExpressionNeedsToResolved(child, isIncludeFilter)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/GenericQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/GenericQueryType.java b/core/src/main/java/org/carbondata/scan/filter/GenericQueryType.java
new file mode 100644
index 0000000..30bb6c2
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/GenericQueryType.java
@@ -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.carbondata.scan.filter;
+
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.List;
+
+import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+
+import org.apache.spark.sql.types.DataType;
+
+public interface GenericQueryType {
+
+  String getName();
+
+  void setName(String name);
+
+  String getParentname();
+
+  void setParentname(String parentname);
+
+  int getBlockIndex();
+
+  void setBlockIndex(int blockIndex);
+
+  void addChildren(GenericQueryType children);
+
+  void getAllPrimitiveChildren(List<GenericQueryType> primitiveChild);
+
+  int getSurrogateIndex();
+
+  void setSurrogateIndex(int surrIndex);
+
+  int getColsCount();
+
+  void setKeySize(int[] keyBlockSize);
+
+  int getKeyOrdinalForQuery();
+
+  void setKeyOrdinalForQuery(int keyOrdinalForQuery);
+
+  void parseBlocksAndReturnComplexColumnByteArray(
+      ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolder, int rowNumber,
+      DataOutputStream dataOutputStream) throws IOException;
+
+  DataType getSchemaType();
+
+  void parseAndGetResultBytes(ByteBuffer complexData, DataOutputStream dataOutput)
+      throws IOException;
+
+  void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/AndFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/AndFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/AndFilterExecuterImpl.java
new file mode 100644
index 0000000..10ad66f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/AndFilterExecuterImpl.java
@@ -0,0 +1,62 @@
+/*
+ * 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.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+
+public class AndFilterExecuterImpl implements FilterExecuter {
+
+  private FilterExecuter leftExecuter;
+  private FilterExecuter rightExecuter;
+
+  public AndFilterExecuterImpl(FilterExecuter leftExecuter, FilterExecuter rightExecuter) {
+    this.leftExecuter = leftExecuter;
+    this.rightExecuter = rightExecuter;
+  }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
+      throws FilterUnsupportedException {
+    BitSet leftFilters = leftExecuter.applyFilter(blockChunkHolder);
+    if (leftFilters.isEmpty()) {
+      return leftFilters;
+    }
+    BitSet rightFilter = rightExecuter.applyFilter(blockChunkHolder);
+    if (rightFilter.isEmpty()) {
+      return rightFilter;
+    }
+    leftFilters.and(rightFilter);
+    return leftFilters;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
+    if (leftFilters.isEmpty()) {
+      return leftFilters;
+    }
+    BitSet rightFilter = rightExecuter.isScanRequired(blockMaxValue, blockMinValue);
+    if (rightFilter.isEmpty()) {
+      return rightFilter;
+    }
+    leftFilters.and(rightFilter);
+    return leftFilters;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/ColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/ColGroupFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/ColGroupFilterExecuterImpl.java
new file mode 100644
index 0000000..187ed7a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/ColGroupFilterExecuterImpl.java
@@ -0,0 +1,190 @@
+/*
+ * 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.carbondata.scan.filter.executer;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.util.QueryUtil;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+/**
+ * It checks if filter is required on given block and if required, it does
+ * linear search on block data and set the bitset.
+ */
+public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ColGroupFilterExecuterImpl.class.getName());
+
+  /**
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   */
+  public ColGroupFilterExecuterImpl(DimColumnResolvedFilterInfo dimColResolvedFilterInfo,
+      SegmentProperties segmentProperties) {
+    super(dimColResolvedFilterInfo, segmentProperties);
+  }
+
+  /**
+   * It fills BitSet with row index which matches filter key
+   */
+  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+
+    try {
+      KeyStructureInfo keyStructureInfo = getKeyStructureInfo();
+      byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+      for (int i = 0; i < filterValues.length; i++) {
+        byte[] filterVal = filterValues[i];
+        for (int rowId = 0; rowId < numerOfRows; rowId++) {
+          byte[] colData = new byte[keyStructureInfo.getMaskByteRanges().length];
+          dimensionColumnDataChunk.fillChunkData(colData, 0, rowId, keyStructureInfo);
+          if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, colData) == 0) {
+            bitSet.set(rowId);
+          }
+        }
+      }
+
+    } catch (Exception e) {
+      LOGGER.error(e);
+    }
+
+    return bitSet;
+  }
+
+  /**
+   * It is required for extracting column data from columngroup chunk
+   *
+   * @return
+   * @throws KeyGenException
+   */
+  private KeyStructureInfo getKeyStructureInfo() throws KeyGenException {
+    List<Integer> ordinals = new ArrayList<Integer>();
+    ordinals.add(dimColumnEvaluatorInfo.getColumnIndex());
+
+    int[] maskByteRanges = QueryUtil
+        .getMaskedByteRangeBasedOrdinal(ordinals, segmentProperties.getDimensionKeyGenerator());
+    byte[] maxKey =
+        QueryUtil.getMaxKeyBasedOnOrinal(ordinals, segmentProperties.getDimensionKeyGenerator());
+    int[] maksedByte = QueryUtil
+        .getMaskedByte(segmentProperties.getDimensionKeyGenerator().getKeySizeInBytes(),
+            maskByteRanges);
+    int blockMdkeyStartOffset = QueryUtil.getBlockMdKeyStartOffset(segmentProperties, ordinals);
+
+    KeyStructureInfo restructureInfos = new KeyStructureInfo();
+    restructureInfos.setKeyGenerator(segmentProperties.getDimensionKeyGenerator());
+    restructureInfos.setMaskByteRanges(maskByteRanges);
+    restructureInfos.setMaxKey(maxKey);
+    restructureInfos.setMaskedBytes(maksedByte);
+    restructureInfos.setBlockMdKeyStartOffset(blockMdkeyStartOffset);
+    return restructureInfos;
+  }
+
+  /**
+   * Check if scan is required on given block based on min and max value
+   */
+  public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
+    BitSet bitSet = new BitSet(1);
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    int columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
+    int[] cols = getAllColumns(columnIndex);
+    byte[] maxValue = getMinMaxData(cols, blkMaxVal[blockIndex], columnIndex);
+    byte[] minValue = getMinMaxData(cols, blkMinVal[blockIndex], columnIndex);
+    boolean isScanRequired = false;
+    for (int k = 0; k < filterValues.length; k++) {
+      // filter value should be in range of max and min value i.e
+      // max>filtervalue>min
+      // so filter-max should be negative
+      int maxCompare = ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], maxValue);
+      // and filter-min should be positive
+      int minCompare = ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], minValue);
+
+      // if any filter value is in range than this block needs to be
+      // scanned
+      if (maxCompare <= 0 && minCompare >= 0) {
+        isScanRequired = true;
+        break;
+      }
+    }
+    if (isScanRequired) {
+      bitSet.set(0);
+    }
+    return bitSet;
+  }
+
+  /**
+   * It extract min and max data for given column from stored min max value
+   *
+   * @param cols
+   * @param minMaxData
+   * @param columnIndex
+   * @return
+   */
+  private byte[] getMinMaxData(int[] colGrpColumns, byte[] minMaxData, int columnIndex) {
+    int startIndex = 0;
+    int endIndex = 0;
+    for (int i = 0; i < colGrpColumns.length; i++) {
+      int[] byteRange =
+          segmentProperties.getDimensionKeyGenerator().getKeyByteOffsets(colGrpColumns[i]);
+      int colSize = 0;
+      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
+        colSize++;
+      }
+      if (colGrpColumns[i] == columnIndex) {
+        endIndex = startIndex + colSize;
+        break;
+      }
+      startIndex += colSize;
+    }
+    byte[] data = new byte[endIndex - startIndex];
+    System.arraycopy(minMaxData, startIndex, data, 0, data.length);
+    return data;
+  }
+
+  /**
+   * It returns column groups which have provided column ordinal
+   *
+   * @param columnIndex
+   * @return column group array
+   */
+  private int[] getAllColumns(int columnIndex) {
+    int[][] colGroups = segmentProperties.getColumnGroups();
+    for (int i = 0; i < colGroups.length; i++) {
+      if (QueryUtil.searchInArray(colGroups[i], columnIndex)) {
+        return colGroups[i];
+      }
+    }
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/DimColumnExecuterFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/DimColumnExecuterFilterInfo.java b/core/src/main/java/org/carbondata/scan/filter/executer/DimColumnExecuterFilterInfo.java
new file mode 100644
index 0000000..a81b4a1
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/DimColumnExecuterFilterInfo.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.carbondata.scan.filter.executer;
+
+public class DimColumnExecuterFilterInfo {
+
+  byte[][] filterKeys;
+
+  public void setFilterKeys(byte[][] filterKeys) {
+    this.filterKeys = filterKeys;
+  }
+
+  public byte[][] getFilterKeys() {
+    return filterKeys;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java
new file mode 100644
index 0000000..6eb7d9a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@ -0,0 +1,205 @@
+/*
+ * 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.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.carbondata.core.keygenerator.KeyGenerator;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+
+public class ExcludeFilterExecuterImpl implements FilterExecuter {
+
+  DimColumnResolvedFilterInfo dimColEvaluatorInfo;
+  DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+
+  public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo) {
+    this.dimColEvaluatorInfo = dimColEvaluatorInfo;
+  }
+
+  public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
+      KeyGenerator blockKeyGenerator) {
+    this(dimColEvaluatorInfo);
+    dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
+    FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(), blockKeyGenerator,
+        dimColEvaluatorInfo.getDimension(), dimColumnExecuterInfo);
+  }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) {
+    if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
+      blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
+          dimColEvaluatorInfo.getColumnIndex());
+    }
+    if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
+      blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()] =
+          blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
+              dimColEvaluatorInfo.getColumnIndex());
+    }
+    return getFilteredIndexes(
+        blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()],
+        blockChunkHolder.getDataBlock().nodeSize());
+  }
+
+  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimColumnDataChunk, int numerOfRows) {
+    // For high cardinality dimensions.
+    if (dimColumnDataChunk.getAttributes().isNoDictionary()
+        && dimColumnDataChunk instanceof VariableLengthDimensionDataChunk) {
+      return setDirectKeyFilterIndexToBitSet((VariableLengthDimensionDataChunk) dimColumnDataChunk,
+          numerOfRows);
+    }
+    if (null != dimColumnDataChunk.getAttributes().getInvertedIndexes()
+        && dimColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      return setFilterdIndexToBitSetWithColumnIndex(
+          (FixedLengthDimensionDataChunk) dimColumnDataChunk, numerOfRows);
+    }
+    return setFilterdIndexToBitSet((FixedLengthDimensionDataChunk) dimColumnDataChunk, numerOfRows);
+  }
+
+  private BitSet setDirectKeyFilterIndexToBitSet(
+      VariableLengthDimensionDataChunk dimColumnDataChunk, int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    bitSet.flip(0, numerOfRows);
+    List<byte[]> listOfColumnarKeyBlockDataForNoDictionaryVal =
+        dimColumnDataChunk.getCompleteDataChunk();
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    int[] columnIndexArray = dimColumnDataChunk.getAttributes().getInvertedIndexes();
+    int[] columnReverseIndexArray = dimColumnDataChunk.getAttributes().getInvertedIndexesReverse();
+    for (int i = 0; i < filterValues.length; i++) {
+      byte[] filterVal = filterValues[i];
+      if (null != listOfColumnarKeyBlockDataForNoDictionaryVal) {
+
+        if (null != columnReverseIndexArray) {
+          for (int index : columnIndexArray) {
+            byte[] noDictionaryVal =
+                listOfColumnarKeyBlockDataForNoDictionaryVal.get(columnReverseIndexArray[index]);
+            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
+              bitSet.flip(index);
+            }
+          }
+        } else if (null != columnIndexArray) {
+
+          for (int index : columnIndexArray) {
+            byte[] noDictionaryVal =
+                listOfColumnarKeyBlockDataForNoDictionaryVal.get(columnIndexArray[index]);
+            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
+              bitSet.flip(index);
+            }
+          }
+        } else {
+          for (int index = 0;
+               index < listOfColumnarKeyBlockDataForNoDictionaryVal.size(); index++) {
+            if (ByteUtil.UnsafeComparer.INSTANCE
+                .compareTo(filterVal, listOfColumnarKeyBlockDataForNoDictionaryVal.get(index))
+                == 0) {
+              bitSet.flip(index);
+            }
+          }
+
+        }
+
+      }
+    }
+    return bitSet;
+
+  }
+
+  private BitSet setFilterdIndexToBitSetWithColumnIndex(
+      FixedLengthDimensionDataChunk dimColumnDataChunk, int numerOfRows) {
+    int[] columnIndex = dimColumnDataChunk.getAttributes().getInvertedIndexes();
+    int startKey = 0;
+    int last = 0;
+    int startIndex = 0;
+    BitSet bitSet = new BitSet(numerOfRows);
+    bitSet.flip(0, numerOfRows);
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    for (int i = 0; i < filterValues.length; i++) {
+      startKey = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
+              filterValues[i]);
+      if (startKey == -1) {
+        continue;
+      }
+      bitSet.flip(columnIndex[startKey]);
+      last = startKey;
+      for (int j = startKey + 1; j < numerOfRows; j++) {
+        if (ByteUtil.UnsafeComparer.INSTANCE
+            .compareTo(dimColumnDataChunk.getCompleteDataChunk(), j * filterValues[i].length,
+                filterValues[i].length, filterValues[i], 0, filterValues[i].length) == 0) {
+          bitSet.flip(columnIndex[j]);
+          last++;
+        } else {
+          break;
+        }
+      }
+      startIndex = last;
+      if (startIndex >= numerOfRows) {
+        break;
+      }
+    }
+    return bitSet;
+  }
+
+  private BitSet setFilterdIndexToBitSet(FixedLengthDimensionDataChunk dimColumnDataChunk,
+      int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    int startKey = 0;
+    int last = 0;
+    bitSet.flip(0, numerOfRows);
+    int startIndex = 0;
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    for (int k = 0; k < filterValues.length; k++) {
+      startKey = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
+              filterValues[k]);
+      if (startKey == -1) {
+        continue;
+      }
+      bitSet.flip(startKey);
+      last = startKey;
+      for (int j = startKey + 1; j < numerOfRows; j++) {
+        if (ByteUtil.UnsafeComparer.INSTANCE
+            .compareTo(dimColumnDataChunk.getCompleteDataChunk(), j * filterValues[k].length,
+                filterValues[k].length, filterValues[k], 0, filterValues[k].length) == 0) {
+          bitSet.flip(j);
+          last++;
+        } else {
+          break;
+        }
+      }
+      startIndex = last;
+      if (startIndex >= numerOfRows) {
+        break;
+      }
+    }
+    return bitSet;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet bitSet = new BitSet(1);
+    bitSet.flip(0, 1);
+    return bitSet;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/FilterExecuter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/FilterExecuter.java b/core/src/main/java/org/carbondata/scan/filter/executer/FilterExecuter.java
new file mode 100644
index 0000000..42f913b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/FilterExecuter.java
@@ -0,0 +1,45 @@
+/*
+ * 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.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+
+public interface FilterExecuter {
+
+  /**
+   * API will apply filter based on resolver instance
+   *
+   * @return
+   * @throws FilterUnsupportedException
+   */
+  BitSet applyFilter(BlocksChunkHolder blocksChunkHolder) throws FilterUnsupportedException;
+
+  /**
+   * API will verify whether the block can be shortlisted based on block
+   * max and min key.
+   *
+   * @param blockMaxValue, maximum value of the
+   * @param blockMinValue
+   * @return BitSet
+   */
+  BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue);
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java
new file mode 100644
index 0000000..71567ea
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java
@@ -0,0 +1,224 @@
+/*
+ * 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.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+
+public class IncludeFilterExecuterImpl implements FilterExecuter {
+
+  protected DimColumnResolvedFilterInfo dimColumnEvaluatorInfo;
+  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+  protected SegmentProperties segmentProperties;
+
+  public IncludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColumnEvaluatorInfo,
+      SegmentProperties segmentProperties) {
+    this.dimColumnEvaluatorInfo = dimColumnEvaluatorInfo;
+    this.segmentProperties = segmentProperties;
+    dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
+    FilterUtil.prepareKeysFromSurrogates(dimColumnEvaluatorInfo.getFilterValues(),
+        segmentProperties.getDimensionKeyGenerator(), dimColumnEvaluatorInfo.getDimension(),
+        dimColumnExecuterInfo);
+
+  }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) {
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+        .get(dimColumnEvaluatorInfo.getColumnIndex());
+    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
+      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
+          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    }
+    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
+        blockChunkHolder.getDataBlock().nodeSize());
+  }
+
+  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    if (dimensionColumnDataChunk.getAttributes().isNoDictionary()
+        && dimensionColumnDataChunk instanceof VariableLengthDimensionDataChunk) {
+      return setDirectKeyFilterIndexToBitSet(
+          (VariableLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+    } else if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
+        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      return setFilterdIndexToBitSetWithColumnIndex(
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+    }
+
+    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+  }
+
+  private BitSet setDirectKeyFilterIndexToBitSet(
+      VariableLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    List<byte[]> listOfColumnarKeyBlockDataForNoDictionaryVals =
+        dimensionColumnDataChunk.getCompleteDataChunk();
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    int[] columnIndexArray = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
+    int[] columnReverseIndexArray =
+        dimensionColumnDataChunk.getAttributes().getInvertedIndexesReverse();
+    for (int i = 0; i < filterValues.length; i++) {
+      byte[] filterVal = filterValues[i];
+      if (null != listOfColumnarKeyBlockDataForNoDictionaryVals) {
+        if (null != columnIndexArray) {
+          for (int index : columnIndexArray) {
+            byte[] noDictionaryVal =
+                listOfColumnarKeyBlockDataForNoDictionaryVals.get(columnReverseIndexArray[index]);
+            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
+              bitSet.set(index);
+            }
+          }
+        } else if (null != columnReverseIndexArray) {
+          for (int index : columnReverseIndexArray) {
+            byte[] noDictionaryVal =
+                listOfColumnarKeyBlockDataForNoDictionaryVals.get(columnReverseIndexArray[index]);
+            if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterVal, noDictionaryVal) == 0) {
+              bitSet.set(index);
+            }
+          }
+        } else {
+          for (int index = 0;
+               index < listOfColumnarKeyBlockDataForNoDictionaryVals.size(); index++) {
+            if (ByteUtil.UnsafeComparer.INSTANCE
+                .compareTo(filterVal, listOfColumnarKeyBlockDataForNoDictionaryVals.get(index))
+                == 0) {
+              bitSet.set(index);
+            }
+          }
+        }
+      }
+    }
+    return bitSet;
+
+  }
+
+  private BitSet setFilterdIndexToBitSetWithColumnIndex(
+      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    int[] columnIndex = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
+    int start = 0;
+    int last = 0;
+    int startIndex = 0;
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    for (int i = 0; i < filterValues.length; i++) {
+      start = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+              filterValues[i]);
+      if (start == -1) {
+        continue;
+      }
+      bitSet.set(columnIndex[start]);
+      last = start;
+      for (int j = start + 1; j < numerOfRows; j++) {
+        if (ByteUtil.UnsafeComparer.INSTANCE
+            .compareTo(dimensionColumnDataChunk.getCompleteDataChunk(), j * filterValues[i].length,
+                filterValues[i].length, filterValues[i], 0, filterValues[i].length) == 0) {
+          bitSet.set(columnIndex[j]);
+          last++;
+        } else {
+          break;
+        }
+      }
+      startIndex = last;
+      if (startIndex >= numerOfRows) {
+        break;
+      }
+    }
+    return bitSet;
+  }
+
+  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      FixedLengthDimensionDataChunk fixedDimensionChunk =
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk;
+      int start = 0;
+      int last = 0;
+      int startIndex = 0;
+      byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+      for (int k = 0; k < filterValues.length; k++) {
+        start = CarbonUtil.getFirstIndexUsingBinarySearch(
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            filterValues[k]);
+        if (start == -1) {
+          continue;
+        }
+        bitSet.set(start);
+        last = start;
+        for (int j = start + 1; j < numerOfRows; j++) {
+          if (ByteUtil.UnsafeComparer.INSTANCE
+              .compareTo(fixedDimensionChunk.getCompleteDataChunk(), j * filterValues[k].length,
+                  filterValues[k].length, filterValues[k], 0, filterValues[k].length) == 0) {
+            bitSet.set(j);
+            last++;
+          } else {
+            break;
+          }
+        }
+        startIndex = last;
+        if (startIndex >= numerOfRows) {
+          break;
+        }
+      }
+    }
+    return bitSet;
+  }
+
+  public BitSet isScanRequired(byte[][] blkMaxVal, byte[][] blkMinVal) {
+    BitSet bitSet = new BitSet(1);
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    int columnIndex = dimColumnEvaluatorInfo.getColumnIndex();
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping().get(columnIndex);
+
+    boolean isScanRequired = false;
+    for (int k = 0; k < filterValues.length; k++) {
+      // filter value should be in range of max and min value i.e
+      // max>filtervalue>min
+      // so filter-max should be negative
+      int maxCompare =
+          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blkMaxVal[blockIndex]);
+      // and filter-min should be positive
+      int minCompare =
+          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blkMinVal[blockIndex]);
+
+      // if any filter value is in range than this block needs to be
+      // scanned
+      if (maxCompare <= 0 && minCompare >= 0) {
+        isScanRequired = true;
+        break;
+      }
+    }
+    if (isScanRequired) {
+      bitSet.set(0);
+    }
+    return bitSet;
+  }
+
+}



[08/56] [abbrv] incubator-carbondata git commit: [issue-644]Update CarbonFile to support Federation (#661)

Posted by jb...@apache.org.
[issue-644]Update CarbonFile to support Federation (#661)

* Update CarbonFile to support Federation

* Set DefaultFileType to VIEWFS when store-path schema is viewfs://

* fix checkstyle error

* fix instance error of CarbonFile#getParentFile return

* update #ViewFSCarbonFile follow of optimizing nn rpc request


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

Branch: refs/heads/master
Commit: 25cd9e5de13e2b9c561308be2a376ec458be5168
Parents: dbefb7b
Author: Hexiaoqiao <xq...@gmail.com>
Authored: Fri Jun 17 11:39:13 2016 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Fri Jun 17 09:09:13 2016 +0530

----------------------------------------------------------------------
 .../fileperations/AtomicFileOperationsImpl.java |   3 +-
 .../store/filesystem/AbstractDFSCarbonFile.java | 217 +++++++++++++++++
 .../store/filesystem/HDFSCarbonFile.java        | 243 +++----------------
 .../store/filesystem/ViewFSCarbonFile.java      | 126 ++++++++++
 .../store/impl/DFSFileHolderImpl.java           | 183 ++++++++++++++
 .../datastorage/store/impl/FileFactory.java     |  36 ++-
 .../store/impl/HDFSFileHolderImpl.java          | 186 --------------
 .../org/carbondata/core/util/CarbonUtil.java    |  23 +-
 .../carbondata/hadoop/util/SchemaReader.java    |   3 +-
 .../csvreaderstep/BlockDataHandler.java         |   3 +-
 .../util/CarbonDataProcessorUtil.java           |  12 +-
 11 files changed, 613 insertions(+), 422 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java b/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java
index fe7a6f3..41f4580 100644
--- a/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/fileperations/AtomicFileOperationsImpl.java
@@ -77,7 +77,8 @@ public class AtomicFileOperationsImpl implements AtomicFileOperations {
       CarbonFile tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
 
       if (!tempFile.renameForce(filePath)) {
-        throw new IOException("temporary file renaming failed");
+        throw new IOException("temporary file renaming failed, src="
+            + tempFile.getPath() + ", dest=" + filePath);
       }
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/AbstractDFSCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/AbstractDFSCarbonFile.java b/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/AbstractDFSCarbonFile.java
new file mode 100644
index 0000000..b04cd47
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/AbstractDFSCarbonFile.java
@@ -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.carbondata.core.datastorage.store.filesystem;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+import org.carbondata.core.util.CarbonUtil;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public abstract  class AbstractDFSCarbonFile implements CarbonFile {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractDFSCarbonFile.class.getName());
+  protected FileStatus fileStatus;
+  protected FileSystem fs;
+
+  public AbstractDFSCarbonFile(String filePath) {
+    filePath = filePath.replace("\\", "/");
+    Path path = new Path(filePath);
+    try {
+      fs = path.getFileSystem(FileFactory.getConfiguration());
+      fileStatus = fs.getFileStatus(path);
+    } catch (IOException e) {
+      LOGGER.error("Exception occured:" + e.getMessage());
+    }
+  }
+
+  public AbstractDFSCarbonFile(Path path) {
+    try {
+      fs = path.getFileSystem(FileFactory.getConfiguration());
+      fileStatus = fs.getFileStatus(path);
+    } catch (IOException e) {
+      LOGGER.error("Exception occured:" + e.getMessage());
+    }
+  }
+
+  public AbstractDFSCarbonFile(FileStatus fileStatus) {
+    this.fileStatus = fileStatus;
+  }
+
+  @Override public boolean createNewFile() {
+    Path path = fileStatus.getPath();
+    try {
+      return fs.createNewFile(path);
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  @Override public String getAbsolutePath() {
+    return fileStatus.getPath().toString();
+  }
+
+  @Override public String getName() {
+    return fileStatus.getPath().getName();
+  }
+
+  @Override public boolean isDirectory() {
+    return fileStatus.isDirectory();
+  }
+
+  @Override public boolean exists() {
+    try {
+      if (null != fileStatus) {
+        fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
+        return fs.exists(fileStatus.getPath());
+      }
+    } catch (IOException e) {
+      LOGGER.error("Exception occured:" + e.getMessage());
+    }
+    return false;
+  }
+
+  @Override public String getCanonicalPath() {
+    return getAbsolutePath();
+  }
+
+  @Override public String getPath() {
+    return getAbsolutePath();
+  }
+
+  @Override public long getSize() {
+    return fileStatus.getLen();
+  }
+
+  public boolean renameTo(String changetoName) {
+    FileSystem fs;
+    try {
+      fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
+      return fs.rename(fileStatus.getPath(), new Path(changetoName));
+    } catch (IOException e) {
+      LOGGER.error("Exception occured:" + e.getMessage());
+      return false;
+    }
+  }
+
+  public boolean delete() {
+    FileSystem fs;
+    try {
+      fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
+      return fs.delete(fileStatus.getPath(), true);
+    } catch (IOException e) {
+      LOGGER.error("Exception occured:" + e.getMessage());
+      return false;
+    }
+  }
+
+  @Override public long getLastModifiedTime() {
+    return fileStatus.getModificationTime();
+  }
+
+  @Override public boolean setLastModifiedTime(long timestamp) {
+    try {
+      fs.setTimes(fileStatus.getPath(), timestamp, timestamp);
+    } catch (IOException e) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * This method will delete the data in file data from a given offset
+   */
+  @Override public boolean truncate(String fileName, long validDataEndOffset) {
+    DataOutputStream dataOutputStream = null;
+    DataInputStream dataInputStream = null;
+    boolean fileTruncatedSuccessfully = false;
+    // if bytes to read less than 1024 then buffer size should be equal to the given offset
+    int bufferSize = validDataEndOffset > CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR ?
+        CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR :
+        (int) validDataEndOffset;
+    // temporary file name
+    String tempWriteFilePath = fileName + CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
+    FileFactory.FileType fileType = FileFactory.getFileType(fileName);
+    try {
+      CarbonFile tempFile = null;
+      // delete temporary file if it already exists at a given path
+      if (FileFactory.isFileExist(tempWriteFilePath, fileType)) {
+        tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
+        tempFile.delete();
+      }
+      // create new temporary file
+      FileFactory.createNewFile(tempWriteFilePath, fileType);
+      tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
+      byte[] buff = new byte[bufferSize];
+      dataInputStream = FileFactory.getDataInputStream(fileName, fileType);
+      // read the data
+      int read = dataInputStream.read(buff, 0, buff.length);
+      dataOutputStream = FileFactory.getDataOutputStream(tempWriteFilePath, fileType);
+      dataOutputStream.write(buff, 0, read);
+      long remaining = validDataEndOffset - read;
+      // anytime we should not cross the offset to be read
+      while (remaining > 0) {
+        if (remaining > bufferSize) {
+          buff = new byte[bufferSize];
+        } else {
+          buff = new byte[(int) remaining];
+        }
+        read = dataInputStream.read(buff, 0, buff.length);
+        dataOutputStream.write(buff, 0, read);
+        remaining = remaining - read;
+      }
+      CarbonUtil.closeStreams(dataInputStream, dataOutputStream);
+      // rename the temp file to original file
+      tempFile.renameForce(fileName);
+      fileTruncatedSuccessfully = true;
+    } catch (IOException e) {
+      LOGGER.error("Exception occured while truncating the file " + e.getMessage());
+    } finally {
+      CarbonUtil.closeStreams(dataOutputStream, dataInputStream);
+    }
+    return fileTruncatedSuccessfully;
+  }
+
+  /**
+   * This method will be used to check whether a file has been modified or not
+   *
+   * @param fileTimeStamp time to be compared with latest timestamp of file
+   * @param endOffset     file length to be compared with current length of file
+   * @return
+   */
+  @Override public boolean isFileModified(long fileTimeStamp, long endOffset) {
+    boolean isFileModified = false;
+    if (getLastModifiedTime() > fileTimeStamp || getSize() > endOffset) {
+      isFileModified = true;
+    }
+    return isFileModified;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/HDFSCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/HDFSCarbonFile.java b/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/HDFSCarbonFile.java
index a4127f4..98e40b4 100644
--- a/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/HDFSCarbonFile.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/HDFSCarbonFile.java
@@ -19,164 +19,36 @@
 
 package org.carbondata.core.datastorage.store.filesystem;
 
-import java.io.DataInputStream;
-import java.io.DataOutputStream;
 import java.io.IOException;
 import java.util.ArrayList;
 import java.util.List;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.util.CarbonUtil;
 
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 
-public class HDFSCarbonFile implements CarbonFile {
+public class HDFSCarbonFile extends AbstractDFSCarbonFile {
   /**
    * LOGGER
    */
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(HDFSCarbonFile.class.getName());
-  private FileStatus fileStatus;
-  private FileSystem fs;
 
   public HDFSCarbonFile(String filePath) {
-    filePath = filePath.replace("\\", "/");
-    Path path = new Path(filePath);
-    try {
-      fs = path.getFileSystem(FileFactory.getConfiguration());
-      fileStatus = fs.getFileStatus(path);
-    } catch (IOException e) {
-      LOGGER.error("Exception occured: " + e.getMessage());
-    }
+    super(filePath);
   }
 
   public HDFSCarbonFile(Path path) {
-    try {
-      fs = path.getFileSystem(FileFactory.getConfiguration());
-      fileStatus = fs.getFileStatus(path);
-    } catch (IOException e) {
-      LOGGER.error("Exception occured: " + e.getMessage());
-    }
+    super(path);
   }
 
   public HDFSCarbonFile(FileStatus fileStatus) {
-    this.fileStatus = fileStatus;
-  }
-
-  @Override public boolean createNewFile() {
-    Path path = fileStatus.getPath();
-    try {
-      return fs.createNewFile(path);
-    } catch (IOException e) {
-      return false;
-    }
-
-  }
-
-  @Override public String getAbsolutePath() {
-    return fileStatus.getPath().toString();
-  }
-
-  @Override public CarbonFile[] listFiles(final CarbonFileFilter fileFilter) {
-    CarbonFile[] files = listFiles();
-    if (files != null && files.length >= 1) {
-
-      List<CarbonFile> fileList = new ArrayList<CarbonFile>(files.length);
-      for (int i = 0; i < files.length; i++) {
-        if (fileFilter.accept(files[i])) {
-          fileList.add(files[i]);
-        }
-      }
-
-      if (fileList.size() >= 1) {
-        return fileList.toArray(new CarbonFile[fileList.size()]);
-      } else {
-        return new CarbonFile[0];
-      }
-    }
-    return files;
-  }
-
-  @Override public String getName() {
-    return fileStatus.getPath().getName();
-  }
-
-  @Override public boolean isDirectory() {
-    return fileStatus.isDirectory();
-  }
-
-  @Override public boolean exists() {
-    try {
-      if (null != fileStatus) {
-        fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
-        return fs.exists(fileStatus.getPath());
-      }
-    } catch (IOException e) {
-      LOGGER.error("Exception occured: " + e.getMessage());
-    }
-    return false;
-  }
-
-  @Override public String getCanonicalPath() {
-    return getAbsolutePath();
-  }
-
-  @Override public CarbonFile getParentFile() {
-    return new HDFSCarbonFile(fileStatus.getPath().getParent());
-  }
-
-  @Override public String getPath() {
-    return getAbsolutePath();
-  }
-
-  @Override public long getSize() {
-    return fileStatus.getLen();
-  }
-
-  public boolean renameTo(String changetoName) {
-    FileSystem fs;
-    try {
-      fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
-      return fs.rename(fileStatus.getPath(), new Path(changetoName));
-    } catch (IOException e) {
-      LOGGER.error("Exception occured: " + e.getMessage());
-      return false;
-    }
-  }
-
-  public boolean delete() {
-    FileSystem fs;
-    try {
-      fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
-      return fs.delete(fileStatus.getPath(), true);
-    } catch (IOException e) {
-      LOGGER.error("Exception occured" + e.getMessage());
-      return false;
-    }
-  }
-
-  @Override public CarbonFile[] listFiles() {
-
-    FileStatus[] listStatus = null;
-    try {
-      if (null != fileStatus && fileStatus.isDirectory()) {
-        Path path = fileStatus.getPath();
-        listStatus = path.getFileSystem(FileFactory.getConfiguration()).listStatus(path);
-      } else {
-        return null;
-      }
-    } catch (IOException ex) {
-      LOGGER.error("Exception occured: " + ex.getMessage());
-      return new CarbonFile[0];
-    }
-
-    return getFiles(listStatus);
+    super(fileStatus);
   }
 
   /**
@@ -187,98 +59,56 @@ public class HDFSCarbonFile implements CarbonFile {
     if (listStatus == null) {
       return new CarbonFile[0];
     }
-
     CarbonFile[] files = new CarbonFile[listStatus.length];
-
     for (int i = 0; i < files.length; i++) {
       files[i] = new HDFSCarbonFile(listStatus[i]);
     }
     return files;
   }
 
-  @Override public long getLastModifiedTime() {
-    return fileStatus.getModificationTime();
-  }
-
-  @Override public boolean setLastModifiedTime(long timestamp) {
+  @Override
+  public CarbonFile[] listFiles() {
+    FileStatus[] listStatus = null;
     try {
-      fs.setTimes(fileStatus.getPath(), timestamp, timestamp);
+      if (null != fileStatus && fileStatus.isDirectory()) {
+        Path path = fileStatus.getPath();
+        listStatus = path.getFileSystem(FileFactory.getConfiguration()).listStatus(path);
+      } else {
+        return null;
+      }
     } catch (IOException e) {
-      return false;
+      LOGGER.error("Exception occured: " + e.getMessage());
+      return new CarbonFile[0];
     }
-    return true;
+    return getFiles(listStatus);
   }
 
-  /**
-   * This method will delete the data in file data from a given offset
-   */
-  @Override public boolean truncate(String fileName, long validDataEndOffset) {
-    DataOutputStream dataOutputStream = null;
-    DataInputStream dataInputStream = null;
-    boolean fileTruncatedSuccessfully = false;
-    // if bytes to read less than 1024 then buffer size should be equal to the given offset
-    int bufferSize = validDataEndOffset > CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR ?
-        CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR :
-        (int) validDataEndOffset;
-    // temporary file name
-    String tempWriteFilePath = fileName + CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
-    FileFactory.FileType fileType = FileFactory.getFileType(fileName);
-    try {
-      CarbonFile tempFile = null;
-      // delete temporary file if it already exists at a given path
-      if (FileFactory.isFileExist(tempWriteFilePath, fileType)) {
-        tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
-        tempFile.delete();
-      }
-      // create new temporary file
-      FileFactory.createNewFile(tempWriteFilePath, fileType);
-      tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
-      byte[] buff = new byte[bufferSize];
-      dataInputStream = FileFactory.getDataInputStream(fileName, fileType);
-      // read the data
-      int read = dataInputStream.read(buff, 0, buff.length);
-      dataOutputStream = FileFactory.getDataOutputStream(tempWriteFilePath, fileType);
-      dataOutputStream.write(buff, 0, read);
-      long remaining = validDataEndOffset - read;
-      // anytime we should not cross the offset to be read
-      while (remaining > 0) {
-        if (remaining > bufferSize) {
-          buff = new byte[bufferSize];
-        } else {
-          buff = new byte[(int) remaining];
+  @Override
+  public CarbonFile[] listFiles(final CarbonFileFilter fileFilter) {
+    CarbonFile[] files = listFiles();
+    if (files != null && files.length >= 1) {
+      List<CarbonFile> fileList = new ArrayList<CarbonFile>(files.length);
+      for (int i = 0; i < files.length; i++) {
+        if (fileFilter.accept(files[i])) {
+          fileList.add(files[i]);
         }
-        read = dataInputStream.read(buff, 0, buff.length);
-        dataOutputStream.write(buff, 0, read);
-        remaining = remaining - read;
       }
-      CarbonUtil.closeStreams(dataInputStream, dataOutputStream);
-      // rename the temp file to original file
-      tempFile.renameForce(fileName);
-      fileTruncatedSuccessfully = true;
-    } catch (IOException e) {
-      LOGGER.error("Exception occured while truncating the file " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(dataOutputStream, dataInputStream);
+      if (fileList.size() >= 1) {
+        return fileList.toArray(new CarbonFile[fileList.size()]);
+      } else {
+        return new CarbonFile[0];
+      }
     }
-    return fileTruncatedSuccessfully;
+    return files;
   }
 
-  /**
-   * This method will be used to check whether a file has been modified or not
-   *
-   * @param fileTimeStamp time to be compared with latest timestamp of file
-   * @param endOffset     file length to be compared with current length of file
-   * @return
-   */
-  @Override public boolean isFileModified(long fileTimeStamp, long endOffset) {
-    boolean isFileModified = false;
-    if (getLastModifiedTime() > fileTimeStamp || getSize() > endOffset) {
-      isFileModified = true;
-    }
-    return isFileModified;
+  @Override
+  public CarbonFile getParentFile() {
+    return new HDFSCarbonFile(fileStatus.getPath().getParent());
   }
 
-  @Override public boolean renameForce(String changetoName) {
+  @Override
+  public boolean renameForce(String changetoName) {
     FileSystem fs;
     try {
       fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
@@ -294,5 +124,4 @@ public class HDFSCarbonFile implements CarbonFile {
       return false;
     }
   }
-
-}
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/ViewFSCarbonFile.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/ViewFSCarbonFile.java b/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/ViewFSCarbonFile.java
new file mode 100644
index 0000000..c7e4497
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/filesystem/ViewFSCarbonFile.java
@@ -0,0 +1,126 @@
+/*
+ * 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.carbondata.core.datastorage.store.filesystem;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.viewfs.ViewFileSystem;
+
+public class ViewFSCarbonFile extends AbstractDFSCarbonFile {
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ViewFSCarbonFile.class.getName());
+
+  public ViewFSCarbonFile(String filePath) {
+    super(filePath);
+  }
+
+  public ViewFSCarbonFile(Path path) {
+    super(path);
+  }
+
+  public ViewFSCarbonFile(FileStatus fileStatus) {
+    super(fileStatus);
+  }
+
+  /**
+   * @param listStatus
+   * @return
+   */
+  private CarbonFile[] getFiles(FileStatus[] listStatus) {
+    if (listStatus == null) {
+      return new CarbonFile[0];
+    }
+    CarbonFile[] files = new CarbonFile[listStatus.length];
+    for (int i = 0; i < files.length; i++) {
+      files[i] = new ViewFSCarbonFile(listStatus[i]);
+    }
+    return files;
+  }
+
+  @Override
+  public CarbonFile[] listFiles() {
+    FileStatus[] listStatus = null;
+    try {
+      if (null != fileStatus && fileStatus.isDirectory()) {
+        Path path = fileStatus.getPath();
+        listStatus = path.getFileSystem(FileFactory.getConfiguration()).listStatus(path);
+      } else {
+        return null;
+      }
+    } catch (IOException ex) {
+      LOGGER.error("Exception occured" + ex.getMessage());
+      return new CarbonFile[0];
+    }
+    return getFiles(listStatus);
+  }
+
+  @Override
+  public CarbonFile[] listFiles(final CarbonFileFilter fileFilter) {
+    CarbonFile[] files = listFiles();
+    if (files != null && files.length >= 1) {
+      List<CarbonFile> fileList = new ArrayList<CarbonFile>(files.length);
+      for (int i = 0; i < files.length; i++) {
+        if (fileFilter.accept(files[i])) {
+          fileList.add(files[i]);
+        }
+      }
+      if (fileList.size() >= 1) {
+        return fileList.toArray(new CarbonFile[fileList.size()]);
+      } else {
+        return new CarbonFile[0];
+      }
+    }
+    return files;
+  }
+
+  @Override
+  public CarbonFile getParentFile() {
+    return new ViewFSCarbonFile(fileStatus.getPath().getParent());
+  }
+
+  @Override
+  public boolean renameForce(String changetoName) {
+    FileSystem fs;
+    try {
+      fs = fileStatus.getPath().getFileSystem(FileFactory.getConfiguration());
+      if (fs instanceof ViewFileSystem) {
+        fs.delete(new Path(changetoName), true);
+        fs.rename(fileStatus.getPath(), new Path(changetoName));
+        return true;
+      } else {
+        return false;
+      }
+    } catch (IOException e) {
+      LOGGER.error("Exception occured" + e.getMessage());
+      return false;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/datastorage/store/impl/DFSFileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/impl/DFSFileHolderImpl.java b/core/src/main/java/org/carbondata/core/datastorage/store/impl/DFSFileHolderImpl.java
new file mode 100644
index 0000000..653c243
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/impl/DFSFileHolderImpl.java
@@ -0,0 +1,183 @@
+/*
+ * 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.carbondata.core.datastorage.store.impl;
+
+import java.io.IOException;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Map.Entry;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.FileHolder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+
+public class DFSFileHolderImpl implements FileHolder {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DFSFileHolderImpl.class.getName());
+  /**
+   * cache to hold filename and its stream
+   */
+  private Map<String, FSDataInputStream> fileNameAndStreamCache;
+
+  public DFSFileHolderImpl() {
+    this.fileNameAndStreamCache =
+        new HashMap<String, FSDataInputStream>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  @Override public byte[] readByteArray(String filePath, long offset, int length) {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    byte[] byteBffer = read(fileChannel, length, offset);
+    return byteBffer;
+  }
+
+  /**
+   * This method will be used to check whether stream is already present in
+   * cache or not for filepath if not present then create it and then add to
+   * cache, other wise get from cache
+   *
+   * @param filePath fully qualified file path
+   * @return channel
+   */
+  private FSDataInputStream updateCache(String filePath) {
+    FSDataInputStream fileChannel = fileNameAndStreamCache.get(filePath);
+    try {
+      if (null == fileChannel) {
+        Path pt = new Path(filePath);
+        FileSystem fs = pt.getFileSystem(new Configuration());
+        fileChannel = fs.open(pt);
+        fileNameAndStreamCache.put(filePath, fileChannel);
+      }
+    } catch (IOException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return fileChannel;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and positon
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @param offset  position
+   * @return byte buffer
+   */
+  private byte[] read(FSDataInputStream channel, int size, long offset) {
+    byte[] byteBffer = new byte[size];
+    try {
+      channel.seek(offset);
+      channel.readFully(byteBffer);
+    } catch (Exception e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return byteBffer;
+  }
+
+  /**
+   * This method will be used to read from file based on number of bytes to be read and positon
+   *
+   * @param channel file channel
+   * @param size    number of bytes
+   * @return byte buffer
+   */
+  private byte[] read(FSDataInputStream channel, int size) {
+    byte[] byteBffer = new byte[size];
+    try {
+      channel.readFully(byteBffer);
+    } catch (Exception e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return byteBffer;
+  }
+
+  @Override public int readInt(String filePath, long offset) {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    int i = -1;
+    try {
+      fileChannel.seek(offset);
+      i = fileChannel.readInt();
+    } catch (IOException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+
+    return i;
+  }
+
+  @Override public long readDouble(String filePath, long offset) {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    long i = -1;
+    try {
+      fileChannel.seek(offset);
+      i = fileChannel.readLong();
+    } catch (IOException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+
+    return i;
+  }
+
+  @Override public void finish() {
+    for (Entry<String, FSDataInputStream> entry : fileNameAndStreamCache.entrySet()) {
+      try {
+        FSDataInputStream channel = entry.getValue();
+        if (null != channel) {
+          channel.close();
+        }
+      } catch (IOException exception) {
+        LOGGER.error(exception, exception.getMessage());
+      }
+    }
+
+  }
+
+  @Override public byte[] readByteArray(String filePath, int length) {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    byte[] byteBffer = read(fileChannel, length);
+    return byteBffer;
+  }
+
+  @Override public long readLong(String filePath, long offset) {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    long i = -1;
+    try {
+      fileChannel.seek(offset);
+      i = fileChannel.readLong();
+    } catch (IOException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return i;
+  }
+
+  @Override public int readInt(String filePath) {
+    FSDataInputStream fileChannel = updateCache(filePath);
+    int i = -1;
+    try {
+      i = fileChannel.readInt();
+    } catch (IOException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    return i;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/datastorage/store/impl/FileFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/impl/FileFactory.java b/core/src/main/java/org/carbondata/core/datastorage/store/impl/FileFactory.java
index 11645fc..c88ade8 100644
--- a/core/src/main/java/org/carbondata/core/datastorage/store/impl/FileFactory.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/impl/FileFactory.java
@@ -32,6 +32,7 @@ import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.filesystem.HDFSCarbonFile;
 import org.carbondata.core.datastorage.store.filesystem.LocalCarbonFile;
+import org.carbondata.core.datastorage.store.filesystem.ViewFSCarbonFile;
 import org.carbondata.core.util.CarbonUtil;
 
 import org.apache.commons.io.FileUtils;
@@ -49,8 +50,10 @@ public final class FileFactory {
   static {
     String property = CarbonUtil.getCarbonStorePath(null, null);
     if (property != null) {
-      if (property.startsWith("hdfs://")) {
+      if (property.startsWith(CarbonUtil.HDFS_PREFIX)) {
         storeDefaultFileType = FileType.HDFS;
+      } else if (property.startsWith(CarbonUtil.VIEWFS_PREFIX)) {
+        storeDefaultFileType = FileType.VIEWFS;
       }
     }
 
@@ -71,7 +74,8 @@ public final class FileFactory {
       case LOCAL:
         return new FileHolderImpl();
       case HDFS:
-        return new HDFSFileHolderImpl();
+      case VIEWFS:
+        return new DFSFileHolderImpl();
       default:
         return new FileHolderImpl();
     }
@@ -80,16 +84,20 @@ public final class FileFactory {
   public static FileType getFileType() {
     String property = CarbonUtil.getCarbonStorePath(null, null);
     if (property != null) {
-      if (property.startsWith("hdfs://")) {
+      if (property.startsWith(CarbonUtil.HDFS_PREFIX)) {
         storeDefaultFileType = FileType.HDFS;
+      } else if (property.startsWith(CarbonUtil.VIEWFS_PREFIX)) {
+        storeDefaultFileType = FileType.VIEWFS;
       }
     }
     return storeDefaultFileType;
   }
 
   public static FileType getFileType(String path) {
-    if (path.startsWith("hdfs://")) {
+    if (path.startsWith(CarbonUtil.HDFS_PREFIX)) {
       return FileType.HDFS;
+    } else if (path.startsWith(CarbonUtil.VIEWFS_PREFIX)) {
+      return FileType.VIEWFS;
     }
     return FileType.LOCAL;
   }
@@ -100,6 +108,8 @@ public final class FileFactory {
         return new LocalCarbonFile(path);
       case HDFS:
         return new HDFSCarbonFile(path);
+      case VIEWFS:
+        return new ViewFSCarbonFile(path);
       default:
         return new LocalCarbonFile(path);
     }
@@ -112,6 +122,7 @@ public final class FileFactory {
       case LOCAL:
         return new DataInputStream(new BufferedInputStream(new FileInputStream(path)));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataInputStream stream = fs.open(pt);
@@ -128,6 +139,7 @@ public final class FileFactory {
       case LOCAL:
         return new DataInputStream(new BufferedInputStream(new FileInputStream(path)));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataInputStream stream = fs.open(pt, bufferSize);
@@ -152,6 +164,7 @@ public final class FileFactory {
     path = path.replace("\\", "/");
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataInputStream stream = fs.open(pt, bufferSize);
@@ -176,6 +189,7 @@ public final class FileFactory {
       case LOCAL:
         return new DataOutputStream(new BufferedOutputStream(new FileOutputStream(path)));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataOutputStream stream = fs.create(pt, true);
@@ -192,6 +206,7 @@ public final class FileFactory {
       case LOCAL:
         return new DataOutputStream(new BufferedOutputStream(new FileOutputStream(path)));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataOutputStream stream = fs.create(pt, replicationFactor);
@@ -209,6 +224,7 @@ public final class FileFactory {
         return new DataOutputStream(
             new BufferedOutputStream(new FileOutputStream(path), bufferSize));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataOutputStream stream = fs.create(pt, true, bufferSize);
@@ -227,6 +243,7 @@ public final class FileFactory {
         return new DataOutputStream(
             new BufferedOutputStream(new FileOutputStream(path, append), bufferSize));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataOutputStream stream = null;
@@ -256,6 +273,7 @@ public final class FileFactory {
         return new DataOutputStream(
             new BufferedOutputStream(new FileOutputStream(path), bufferSize));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataOutputStream stream =
@@ -280,6 +298,7 @@ public final class FileFactory {
     filePath = filePath.replace("\\", "/");
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         Path path = new Path(filePath);
         FileSystem fs = path.getFileSystem(configuration);
         if (performFileCheck) {
@@ -311,6 +330,7 @@ public final class FileFactory {
     filePath = filePath.replace("\\", "/");
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         Path path = new Path(filePath);
         FileSystem fs = path.getFileSystem(configuration);
         return fs.exists(path);
@@ -326,6 +346,7 @@ public final class FileFactory {
     filePath = filePath.replace("\\", "/");
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         Path path = new Path(filePath);
         FileSystem fs = path.getFileSystem(configuration);
         return fs.createNewFile(path);
@@ -341,6 +362,7 @@ public final class FileFactory {
     filePath = filePath.replace("\\", "/");
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         Path path = new Path(filePath);
         FileSystem fs = path.getFileSystem(configuration);
         return fs.mkdirs(path);
@@ -366,6 +388,7 @@ public final class FileFactory {
       case LOCAL:
         return new DataOutputStream(new BufferedOutputStream(new FileOutputStream(path, true)));
       case HDFS:
+      case VIEWFS:
         Path pt = new Path(path);
         FileSystem fs = pt.getFileSystem(configuration);
         FSDataOutputStream stream = fs.append(pt);
@@ -388,6 +411,7 @@ public final class FileFactory {
     filePath = filePath.replace("\\", "/");
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         Path path = new Path(filePath);
         FileSystem fs = path.getFileSystem(configuration);
         if (fs.createNewFile(path)) {
@@ -403,7 +427,7 @@ public final class FileFactory {
   }
 
   public enum FileType {
-    LOCAL, HDFS
+    LOCAL, HDFS, VIEWFS
   }
 
   /**
@@ -418,6 +442,7 @@ public final class FileFactory {
     FileType fileType = getFileType(filePath);
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         return filePath;
       case LOCAL:
       default:
@@ -438,6 +463,7 @@ public final class FileFactory {
     FileType fileType = getFileType(filePath);
     switch (fileType) {
       case HDFS:
+      case VIEWFS:
         Path path = new Path(filePath);
         FileSystem fs = path.getFileSystem(configuration);
         return fs.getContentSummary(path).getLength();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/datastorage/store/impl/HDFSFileHolderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/impl/HDFSFileHolderImpl.java b/core/src/main/java/org/carbondata/core/datastorage/store/impl/HDFSFileHolderImpl.java
deleted file mode 100644
index 6e753e6..0000000
--- a/core/src/main/java/org/carbondata/core/datastorage/store/impl/HDFSFileHolderImpl.java
+++ /dev/null
@@ -1,186 +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.carbondata.core.datastorage.store.impl;
-
-import java.io.IOException;
-import java.util.HashMap;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.FileHolder;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataInputStream;
-import org.apache.hadoop.fs.FileSystem;
-import org.apache.hadoop.fs.Path;
-
-public class HDFSFileHolderImpl implements FileHolder {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(HDFSFileHolderImpl.class.getName());
-  /**
-   * cache to hold filename and its stream
-   */
-  private Map<String, FSDataInputStream> fileNameAndStreamCache;
-
-  public HDFSFileHolderImpl() {
-    this.fileNameAndStreamCache =
-        new HashMap<String, FSDataInputStream>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  @Override public byte[] readByteArray(String filePath, long offset, int length) {
-
-    FSDataInputStream fileChannel = updateCache(filePath);
-    byte[] byteBffer = read(fileChannel, length, offset);
-    return byteBffer;
-  }
-
-  /**
-   * This method will be used to check whether stream is already present in
-   * cache or not for filepath if not present then create it and then add to
-   * cache, other wise get from cache
-   *
-   * @param filePath fully qualified file path
-   * @return channel
-   */
-  private FSDataInputStream updateCache(String filePath) {
-    FSDataInputStream fileChannel = fileNameAndStreamCache.get(filePath);
-    try {
-      if (null == fileChannel) {
-        Path pt = new Path(filePath);
-        FileSystem fs = pt.getFileSystem(new Configuration());
-        fileChannel = fs.open(pt);
-        fileNameAndStreamCache.put(filePath, fileChannel);
-      }
-    } catch (IOException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    return fileChannel;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and positon
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @param offset  position
-   * @return byte buffer
-   */
-  private byte[] read(FSDataInputStream channel, int size, long offset) {
-    byte[] byteBffer = new byte[size];
-    try {
-      channel.seek(offset);
-      channel.readFully(byteBffer);
-    } catch (Exception e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    return byteBffer;
-  }
-
-  /**
-   * This method will be used to read from file based on number of bytes to be read and positon
-   *
-   * @param channel file channel
-   * @param size    number of bytes
-   * @return byte buffer
-   */
-  private byte[] read(FSDataInputStream channel, int size) {
-    byte[] byteBffer = new byte[size];
-    try {
-      channel.readFully(byteBffer);
-    } catch (Exception e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    return byteBffer;
-  }
-
-  @Override public int readInt(String filePath, long offset) {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    int i = -1;
-    try {
-      fileChannel.seek(offset);
-      i = fileChannel.readInt();
-    } catch (IOException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-
-    return i;
-  }
-
-  @Override public long readDouble(String filePath, long offset) {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    long i = -1;
-    try {
-      fileChannel.seek(offset);
-      i = fileChannel.readLong();
-    } catch (IOException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-
-    return i;
-  }
-
-  @Override public void finish() {
-    for (Entry<String, FSDataInputStream> entry : fileNameAndStreamCache.entrySet()) {
-      try {
-        FSDataInputStream channel = entry.getValue();
-        if (null != channel) {
-          channel.close();
-        }
-      } catch (IOException exception) {
-        LOGGER.error(exception, exception.getMessage());
-      }
-    }
-
-  }
-
-  @Override public byte[] readByteArray(String filePath, int length) {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    byte[] byteBffer = read(fileChannel, length);
-    return byteBffer;
-  }
-
-  @Override public long readLong(String filePath, long offset) {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    long i = -1;
-    try {
-      fileChannel.seek(offset);
-      i = fileChannel.readLong();
-    } catch (IOException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    return i;
-  }
-
-  @Override public int readInt(String filePath) {
-    FSDataInputStream fileChannel = updateCache(filePath);
-    int i = -1;
-    try {
-      i = fileChannel.readInt();
-    } catch (IOException e) {
-      LOGGER.error(e, e.getMessage());
-    }
-    return i;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 2772362..2910105 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -81,8 +81,8 @@ import org.pentaho.di.core.exception.KettleException;
 
 public final class CarbonUtil {
 
-  private static final String HDFS_PREFIX = "hdfs://";
-
+  public static final String HDFS_PREFIX = "hdfs://";
+  public static final String VIEWFS_PREFIX = "viewfs://";
   private static final String FS_DEFAULT_FS = "fs.defaultFS";
 
   /**
@@ -1213,21 +1213,22 @@ public final class CarbonUtil {
   public static String checkAndAppendHDFSUrl(String filePath) {
     String currentPath = filePath;
     if (null != filePath && filePath.length() != 0 &&
-        FileFactory.getFileType(filePath) != FileFactory.FileType.HDFS) {
-      String baseHDFSUrl = CarbonProperties.getInstance()
+        FileFactory.getFileType(filePath) != FileFactory.FileType.HDFS &&
+        FileFactory.getFileType(filePath) != FileFactory.FileType.VIEWFS) {
+      String baseDFSUrl = CarbonProperties.getInstance()
           .getProperty(CarbonCommonConstants.CARBON_DDL_BASE_HDFS_URL);
-      if (null != baseHDFSUrl) {
-        String hdfsUrl = conf.get(FS_DEFAULT_FS);
-        if (hdfsUrl.startsWith(HDFS_PREFIX)) {
-          baseHDFSUrl = hdfsUrl + baseHDFSUrl;
+      if (null != baseDFSUrl) {
+        String dfsUrl = conf.get(FS_DEFAULT_FS);
+        if (dfsUrl.startsWith(HDFS_PREFIX) || dfsUrl.startsWith(VIEWFS_PREFIX)) {
+          baseDFSUrl = dfsUrl + baseDFSUrl;
         }
-        if (baseHDFSUrl.endsWith("/")) {
-          baseHDFSUrl = baseHDFSUrl.substring(0, baseHDFSUrl.length() - 1);
+        if (baseDFSUrl.endsWith("/")) {
+          baseDFSUrl = baseDFSUrl.substring(0, baseDFSUrl.length() - 1);
         }
         if (!filePath.startsWith("/")) {
           filePath = "/" + filePath;
         }
-        currentPath = baseHDFSUrl + filePath;
+        currentPath = baseDFSUrl + filePath;
       }
     }
     return currentPath;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/hadoop/src/main/java/org/carbondata/hadoop/util/SchemaReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/util/SchemaReader.java b/hadoop/src/main/java/org/carbondata/hadoop/util/SchemaReader.java
index 7f9e56d..31b0cfc 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/util/SchemaReader.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/util/SchemaReader.java
@@ -22,7 +22,8 @@ public class SchemaReader {
   public CarbonTable readCarbonTableFromStore(CarbonTablePath carbonTablePath,
       CarbonTableIdentifier tableIdentifier, String storePath) throws IOException {
     String schemaFilePath = carbonTablePath.getSchemaFilePath();
-    if (FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.HDFS)) {
+    if (FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.HDFS)
+          || FileFactory.isFileExist(schemaFilePath, FileFactory.FileType.VIEWFS)) {
       String tableName = tableIdentifier.getTableName();
 
       ThriftReader.TBaseCreator createTBase = new ThriftReader.TBaseCreator() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java b/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
index 575fd6a..d832504 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
@@ -177,7 +177,8 @@ public class BlockDataHandler {
       }
 
       // Open the next one...
-      if (FileFactory.getFileType(blockDetails.getFilePath()) == FileFactory.FileType.HDFS) {
+      if (FileFactory.getFileType(blockDetails.getFilePath()) == FileFactory.FileType.HDFS
+            || FileFactory.getFileType(blockDetails.getFilePath()) == FileFactory.FileType.VIEWFS) {
         //when case HDFS file type, we use the file path directly
         //give 0 offset as the file start offset when open a new file
         initializeFileReader(blockDetails.getFilePath(), 0);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/25cd9e5d/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
index 4c361fc..a64a256 100644
--- a/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -31,10 +31,7 @@ import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
-import org.carbondata.core.datastorage.store.filesystem.HDFSCarbonFile;
-import org.carbondata.core.datastorage.store.filesystem.LocalCarbonFile;
+import org.carbondata.core.datastorage.store.filesystem.*;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.carbondata.core.load.LoadMetadataDetails;
@@ -125,12 +122,7 @@ public final class CarbonDataProcessorUtil {
     } catch (IOException e1) {
       LOGGER.info("bad record folder does not exist");
     }
-    CarbonFile carbonFile = null;
-    if (fileType.equals(FileFactory.FileType.HDFS)) {
-      carbonFile = new HDFSCarbonFile(badLogStoreLocation);
-    } else {
-      carbonFile = new LocalCarbonFile(badLogStoreLocation);
-    }
+    CarbonFile carbonFile = FileFactory.getCarbonFile(badLogStoreLocation, fileType);
 
     CarbonFile[] listFiles = carbonFile.listFiles(new CarbonFileFilter() {
       @Override public boolean accept(CarbonFile pathname) {


[28/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/OrFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/OrFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/OrFilterExecuterImpl.java
new file mode 100644
index 0000000..38938e6
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/OrFilterExecuterImpl.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+
+public class OrFilterExecuterImpl implements FilterExecuter {
+
+  private FilterExecuter leftExecuter;
+  private FilterExecuter rightExecuter;
+
+  public OrFilterExecuterImpl(FilterExecuter leftExecuter, FilterExecuter rightExecuter) {
+    this.leftExecuter = leftExecuter;
+    this.rightExecuter = rightExecuter;
+  }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
+      throws FilterUnsupportedException {
+    BitSet leftFilters = leftExecuter.applyFilter(blockChunkHolder);
+    BitSet rightFilters = rightExecuter.applyFilter(blockChunkHolder);
+    leftFilters.or(rightFilters);
+
+    return leftFilters;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet leftFilters = leftExecuter.isScanRequired(blockMaxValue, blockMinValue);
+    BitSet rightFilters = rightExecuter.isScanRequired(blockMaxValue, blockMinValue);
+    leftFilters.or(rightFilters);
+    return leftFilters;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/RestructureFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/RestructureFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/RestructureFilterExecuterImpl.java
new file mode 100644
index 0000000..70a6ff0
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RestructureFilterExecuterImpl.java
@@ -0,0 +1,55 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+
+import org.carbondata.core.keygenerator.KeyGenerator;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+
+
+public class RestructureFilterExecuterImpl implements FilterExecuter {
+
+  DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+
+  public RestructureFilterExecuterImpl(DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo,
+      KeyGenerator blockKeyGenerator) {
+    dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
+    FilterUtil
+        .prepareKeysFromSurrogates(dimColumnResolvedFilterInfo.getFilterValues(), blockKeyGenerator,
+            dimColumnResolvedFilterInfo.getDimension(), dimColumnExecuterInfo);
+  }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blocksChunkHolder) {
+    BitSet bitSet = new BitSet(blocksChunkHolder.getDataBlock().nodeSize());
+    byte[][] filterValues = dimColumnExecuterInfo.getFilterKeys();
+    if (null != filterValues && filterValues.length > 0) {
+      bitSet.set(0, blocksChunkHolder.getDataBlock().nodeSize());
+    }
+    return bitSet;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet bitSet = new BitSet(1);
+    bitSet.set(0);
+    return bitSet;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
new file mode 100644
index 0000000..0463978
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@ -0,0 +1,331 @@
+/*
+ * 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.carbondata.scan.filter.executer;
+
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.cache.dictionary.Dictionary;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.chunk.impl.VariableLengthDimensionDataChunk;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.GenericQueryType;
+import org.carbondata.scan.filter.intf.RowImpl;
+import org.carbondata.scan.filter.intf.RowIntf;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+import org.carbondata.scan.processor.BlocksChunkHolder;
+import org.carbondata.scan.util.DataTypeUtil;
+
+public class RowLevelFilterExecuterImpl implements FilterExecuter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(RowLevelFilterExecuterImpl.class.getName());
+  protected List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
+  protected List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
+  protected Expression exp;
+  protected AbsoluteTableIdentifier tableIdentifier;
+
+  public RowLevelFilterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
+      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
+      AbsoluteTableIdentifier tableIdentifier) {
+    this.dimColEvaluatorInfoList = dimColEvaluatorInfoList;
+    if (null == msrColEvalutorInfoList) {
+      this.msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(20);
+    } else {
+      this.msrColEvalutorInfoList = msrColEvalutorInfoList;
+    }
+    this.exp = exp;
+    this.tableIdentifier = tableIdentifier;
+  }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
+      throws FilterUnsupportedException {
+    for (DimColumnResolvedFilterInfo dimColumnEvaluatorInfo : dimColEvaluatorInfoList) {
+      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.ARRAY
+          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.STRUCT) {
+        if (null == blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo
+            .getColumnIndex()]) {
+          blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()] =
+              blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
+                  dimColumnEvaluatorInfo.getColumnIndex());
+        }
+      } else {
+        GenericQueryType complexType = dimColumnEvaluatorInfo.getComplexTypesWithBlockStartIndex()
+            .get(dimColumnEvaluatorInfo.getColumnIndex());
+        complexType.fillRequiredBlockData(blockChunkHolder);
+      }
+    }
+
+    // CHECKSTYLE:OFF Approval No:Approval-V1R2C10_001
+    if (null != msrColEvalutorInfoList) {
+      for (MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo : msrColEvalutorInfoList) {
+        if (msrColumnEvalutorInfo.isMeasureExistsInCurrentSlice() && null == blockChunkHolder
+            .getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]) {
+          blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()] =
+              blockChunkHolder.getDataBlock().getMeasureChunk(blockChunkHolder.getFileReader(),
+                  msrColumnEvalutorInfo.getColumnIndex());
+        }
+      }
+    }
+    // CHECKSTYLE:ON
+
+    int numberOfRows = blockChunkHolder.getDataBlock().nodeSize();
+    BitSet set = new BitSet(numberOfRows);
+    RowIntf row = new RowImpl();
+
+    // CHECKSTYLE:OFF Approval No:Approval-V1R2C10_007
+    for (int index = 0; index < numberOfRows; index++) {
+      try {
+        createRow(blockChunkHolder, row, index);
+      } catch (QueryExecutionException e1) {
+        // TODO Auto-generated catch block
+        e1.printStackTrace();
+      }
+      try {
+        Boolean rslt = exp.evaluate(row).getBoolean();
+        if (null != rslt && rslt) {
+          set.set(index);
+        }
+      } catch (FilterUnsupportedException e) {
+        throw new FilterUnsupportedException(e.getMessage());
+      }
+    }
+    // CHECKSTYLE:ON
+
+    return set;
+  }
+
+  /**
+   * Method will read the members of particular dimension block and create
+   * a row instance for further processing of the filters
+   *
+   * @param blockChunkHolder
+   * @param row
+   * @param index
+   * @throws QueryExecutionException
+   */
+  private void createRow(BlocksChunkHolder blockChunkHolder, RowIntf row, int index)
+      throws QueryExecutionException {
+    Object[] record = new Object[dimColEvaluatorInfoList.size() + msrColEvalutorInfoList.size()];
+    String memberString = null;
+    for (DimColumnResolvedFilterInfo dimColumnEvaluatorInfo : dimColEvaluatorInfoList) {
+      if (dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.ARRAY
+          && dimColumnEvaluatorInfo.getDimension().getDataType() != DataType.STRUCT) {
+        if (!dimColumnEvaluatorInfo.isDimensionExistsInCurrentSilce()) {
+          record[dimColumnEvaluatorInfo.getRowIndex()] = dimColumnEvaluatorInfo.getDefaultValue();
+        }
+        if (!dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
+            && blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo
+            .getColumnIndex()] instanceof VariableLengthDimensionDataChunk) {
+
+          VariableLengthDimensionDataChunk dimensionColumnDataChunk =
+              (VariableLengthDimensionDataChunk) blockChunkHolder
+                  .getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()];
+          if (null != dimensionColumnDataChunk.getCompleteDataChunk()) {
+            memberString =
+                readMemberBasedOnNoDictionaryVal(dimColumnEvaluatorInfo, dimensionColumnDataChunk,
+                    index);
+            if (null != memberString) {
+              if (memberString.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+                memberString = null;
+              }
+            }
+            record[dimColumnEvaluatorInfo.getRowIndex()] = DataTypeUtil
+                .getDataBasedOnDataType(memberString,
+                    dimColumnEvaluatorInfo.getDimension().getDataType());
+          } else {
+            continue;
+          }
+        } else {
+          int dictionaryValue =
+              readSurrogatesFromColumnBlock(blockChunkHolder, index, dimColumnEvaluatorInfo);
+          Dictionary forwardDictionary = null;
+          if (dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
+              && !dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+            memberString =
+                getFilterActualValueFromDictionaryValue(dimColumnEvaluatorInfo, dictionaryValue,
+                    forwardDictionary);
+            record[dimColumnEvaluatorInfo.getRowIndex()] = DataTypeUtil
+                .getDataBasedOnDataType(memberString,
+                    dimColumnEvaluatorInfo.getDimension().getDataType());
+          } else if (dimColumnEvaluatorInfo.getDimension()
+              .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+
+            Object member = getFilterActualValueFromDirectDictionaryValue(dimColumnEvaluatorInfo,
+                dictionaryValue);
+            record[dimColumnEvaluatorInfo.getRowIndex()] = member;
+          }
+        }
+      }
+    }
+
+    DataType msrType;
+
+    for (MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo : msrColEvalutorInfoList) {
+      switch (msrColumnEvalutorInfo.getType()) {
+        case LONG:
+          msrType = DataType.LONG;
+          break;
+        case DECIMAL:
+          msrType = DataType.DECIMAL;
+          break;
+        default:
+          msrType = DataType.DOUBLE;
+      }
+      // if measure doesnt exist then set the default value.
+      if (!msrColumnEvalutorInfo.isMeasureExistsInCurrentSlice()) {
+        record[msrColumnEvalutorInfo.getRowIndex()] = msrColumnEvalutorInfo.getDefaultValue();
+      } else {
+        Object msrValue;
+        switch (msrType) {
+          case LONG:
+            msrValue =
+                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
+                    .getMeasureDataHolder().getReadableLongValueByIndex(index);
+            break;
+          case DECIMAL:
+            msrValue =
+                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
+                    .getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+            break;
+          default:
+            msrValue =
+                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
+                    .getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+        }
+        record[msrColumnEvalutorInfo.getRowIndex()] = msrValue;
+
+      }
+    }
+    row.setValues(record);
+  }
+
+  /**
+   * method will read the actual data from the direct dictionary generator
+   * by passing direct dictionary value.
+   *
+   * @param dimColumnEvaluatorInfo
+   * @param dictionaryValue
+   * @return
+   */
+  private Object getFilterActualValueFromDirectDictionaryValue(
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int dictionaryValue) {
+    Object memberString = null;
+    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+        .getDirectDictionaryGenerator(dimColumnEvaluatorInfo.getDimension().getDataType());
+    if (null != directDictionaryGenerator) {
+      memberString = directDictionaryGenerator.getValueFromSurrogate(dictionaryValue);
+    }
+    return memberString;
+  }
+
+  /**
+   * Read the actual filter member by passing the dictionary value from
+   * the forward dictionary cache which which holds column wise cache
+   *
+   * @param dimColumnEvaluatorInfo
+   * @param dictionaryValue
+   * @param forwardDictionary
+   * @return
+   * @throws QueryExecutionException
+   */
+  private String getFilterActualValueFromDictionaryValue(
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int dictionaryValue,
+      Dictionary forwardDictionary) throws QueryExecutionException {
+    String memberString;
+    try {
+      forwardDictionary = FilterUtil
+          .getForwardDictionaryCache(tableIdentifier, dimColumnEvaluatorInfo.getDimension());
+    } catch (QueryExecutionException e) {
+      throw new QueryExecutionException(e);
+    }
+
+    memberString = forwardDictionary.getDictionaryValueForKey(dictionaryValue);
+    if (null != memberString) {
+      if (memberString.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+        memberString = null;
+      }
+    }
+    return memberString;
+  }
+
+  /**
+   * read the filter member dictionary data from the block corresponding to
+   * applied filter column
+   *
+   * @param blockChunkHolder
+   * @param index
+   * @param dimColumnEvaluatorInfo
+   * @return
+   */
+  private int readSurrogatesFromColumnBlock(BlocksChunkHolder blockChunkHolder, int index,
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo) {
+    byte[] rawData =
+        blockChunkHolder.getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()]
+            .getChunkData(index);
+    ByteBuffer byteBuffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
+    int dictionaryValue = CarbonUtil.getSurrogateKey(rawData, byteBuffer);
+    return dictionaryValue;
+  }
+
+  /**
+   * Reading the blocks for no dictionary data, in no dictionary case
+   * directly the filter data will read, no need to scan the dictionary
+   * or read the dictionary value.
+   *
+   * @param dimColumnEvaluatorInfo
+   * @param dimensionColumnDataChunk
+   * @param index
+   * @return
+   */
+  private String readMemberBasedOnNoDictionaryVal(
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo,
+      VariableLengthDimensionDataChunk dimensionColumnDataChunk, int index) {
+    byte[] noDictionaryVals;
+    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexesReverse()) {
+      // Getting the data for direct surrogates.
+      noDictionaryVals = dimensionColumnDataChunk.getCompleteDataChunk()
+          .get(dimensionColumnDataChunk.getAttributes().getInvertedIndexesReverse()[index]);
+    } else {
+      noDictionaryVals = dimensionColumnDataChunk.getCompleteDataChunk().get(index);
+    }
+    return new String(noDictionaryVals, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet bitSet = new BitSet(1);
+    bitSet.set(0);
+    return bitSet;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
new file mode 100644
index 0000000..2681ccc
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+
+public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
+  private byte[][] filterRangeValues;
+
+  public RowLevelRangeGrtThanFiterExecuterImpl(
+      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
+      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    this.filterRangeValues = filterRangeValues;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet bitSet = new BitSet(1);
+    byte[][] filterValues = this.filterRangeValues;
+    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
+    boolean isScanRequired = false;
+    for (int k = 0; k < filterValues.length; k++) {
+      // filter value should be in range of max and min value i.e
+      // max>filtervalue>min
+      // so filter-max should be negative
+      int maxCompare =
+          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMaxValue[columnIndex]);
+
+      // if any filter value is in range than this block needs to be
+      // scanned means always less than block max range.
+      if (maxCompare < 0) {
+        isScanRequired = true;
+        break;
+      }
+    }
+    if (isScanRequired) {
+      bitSet.set(0);
+    }
+    return bitSet;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
new file mode 100644
index 0000000..ef11b4a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+
+public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl {
+
+  private byte[][] filterRangeValues;
+
+  public RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
+      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
+      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    this.filterRangeValues = filterRangeValues;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet bitSet = new BitSet(1);
+    byte[][] filterValues = this.filterRangeValues;
+    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
+    boolean isScanRequired = false;
+    for (int k = 0; k < filterValues.length; k++) {
+      // filter value should be in range of max and min value i.e
+      // max>filtervalue>min
+      // so filter-max should be negative
+      int maxCompare =
+          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMaxValue[columnIndex]);
+      // if any filter value is in range than this block needs to be
+      // scanned less than equal to max range.
+      if (maxCompare <= 0) {
+        isScanRequired = true;
+        break;
+      }
+    }
+    if (isScanRequired) {
+      bitSet.set(0);
+    }
+    return bitSet;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
new file mode 100644
index 0000000..afc1ccb
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+
+public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilterExecuterImpl {
+  private byte[][] filterRangeValues;
+
+  public RowLevelRangeLessThanEqualFilterExecuterImpl(
+      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
+      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    this.filterRangeValues = filterRangeValues;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet bitSet = new BitSet(1);
+    byte[][] filterValues = this.filterRangeValues;
+    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
+    boolean isScanRequired = false;
+    for (int k = 0; k < filterValues.length; k++) {
+      // and filter-min should be positive
+      int minCompare =
+          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMinValue[columnIndex]);
+
+      // if any filter applied is not in range of min and max of block
+      // then since its a less than equal to fiter validate whether the block
+      // min range is less than equal to applied filter member
+      if (minCompare >= 0) {
+        isScanRequired = true;
+        break;
+      }
+    }
+    if (isScanRequired) {
+      bitSet.set(0);
+    }
+    return bitSet;
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
new file mode 100644
index 0000000..d608cc6
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -0,0 +1,65 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.executer;
+
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+
+public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
+  private byte[][] filterRangeValues;
+
+  public RowLevelRangeLessThanFiterExecuterImpl(
+      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
+      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    this.filterRangeValues = filterRangeValues;
+  }
+
+  @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
+    BitSet bitSet = new BitSet(1);
+    byte[][] filterValues = this.filterRangeValues;
+    int columnIndex = this.dimColEvaluatorInfoList.get(0).getColumnIndex();
+    boolean isScanRequired = false;
+    for (int k = 0; k < filterValues.length; k++) {
+      // and filter-min should be positive
+      int minCompare =
+          ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterValues[k], blockMinValue[columnIndex]);
+
+      // if any filter applied is not in range of min and max of block
+      // then since its a less than fiter validate whether the block
+      // min range is less  than applied filter member
+      if (minCompare > 0) {
+        isScanRequired = true;
+        break;
+      }
+    }
+    if (isScanRequired) {
+      bitSet.set(0);
+    }
+    return bitSet;
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
new file mode 100644
index 0000000..db2fa2d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.executer;
+
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+import org.carbondata.scan.filter.resolver.RowLevelRangeFilterResolverImpl;
+
+public class RowLevelRangeTypeExecuterFacory {
+
+  private RowLevelRangeTypeExecuterFacory() {
+
+  }
+
+  /**
+   * The method returns the Row Level Range fiter type instance based on
+   * filter tree resolver type.
+   *
+   * @param filterExpressionResolverTree
+   * @param dataType                     DataType
+   * @return the generator instance
+   */
+  public static RowLevelFilterExecuterImpl getRowLevelRangeTypeExecuter(
+      FilterExecuterType filterExecuterType, FilterResolverIntf filterExpressionResolverTree) {
+    switch (filterExecuterType) {
+
+      case ROWLEVEL_LESSTHAN:
+        return new RowLevelRangeLessThanFiterExecuterImpl(
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getDimColEvaluatorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getMsrColEvalutorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getFilterRangeValues());
+      case ROWLEVEL_LESSTHAN_EQUALTO:
+        return new RowLevelRangeLessThanEqualFilterExecuterImpl(
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getDimColEvaluatorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getMsrColEvalutorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getFilterRangeValues());
+      case ROWLEVEL_GREATERTHAN_EQUALTO:
+        return new RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getDimColEvaluatorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getMsrColEvalutorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getFilterRangeValues());
+      case ROWLEVEL_GREATERTHAN:
+        return new RowLevelRangeGrtThanFiterExecuterImpl(
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getDimColEvaluatorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getMsrColEvalutorInfoList(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
+                .getFilterRangeValues());
+      default:
+        // Scenario wont come logic must break
+        return null;
+
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/intf/ExpressionType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/intf/ExpressionType.java b/core/src/main/java/org/carbondata/scan/filter/intf/ExpressionType.java
new file mode 100644
index 0000000..2459b5a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/intf/ExpressionType.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.carbondata.scan.filter.intf;
+
+public enum ExpressionType {
+
+  AND,
+  OR,
+  NOT,
+  EQUALS,
+  NOT_EQUALS,
+  LESSTHAN,
+  LESSTHAN_EQUALTO,
+  GREATERTHAN,
+  GREATERTHAN_EQUALTO,
+  ADD,
+  SUBSTRACT,
+  DIVIDE,
+  MULTIPLY,
+  IN,
+  LIST,
+  NOT_IN,
+  UNKNOWN,
+  LITERAL,
+  RANGE
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/intf/FilterExecuterType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/intf/FilterExecuterType.java b/core/src/main/java/org/carbondata/scan/filter/intf/FilterExecuterType.java
new file mode 100644
index 0000000..2de575e
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/intf/FilterExecuterType.java
@@ -0,0 +1,28 @@
+/*
+ * 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.carbondata.scan.filter.intf;
+
+import java.io.Serializable;
+
+public enum FilterExecuterType implements Serializable {
+
+  INCLUDE, EXCLUDE, OR, AND, RESTRUCTURE, ROWLEVEL, RANGE, ROWLEVEL_GREATERTHAN,
+  ROWLEVEL_GREATERTHAN_EQUALTO, ROWLEVEL_LESSTHAN_EQUALTO, ROWLEVEL_LESSTHAN
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/intf/RowImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/intf/RowImpl.java b/core/src/main/java/org/carbondata/scan/filter/intf/RowImpl.java
new file mode 100644
index 0000000..04e1a3d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/intf/RowImpl.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.carbondata.scan.filter.intf;
+
+public class RowImpl implements RowIntf {
+  private Object[] row;
+
+  public RowImpl() {
+    row = new Object[0];
+  }
+
+  @Override public Object getVal(int index) {
+    return row[index];
+  }
+
+  @Override public Object[] getValues() {
+    return row;
+  }
+
+  @Override public void setValues(final Object[] row) {
+    this.row = row;
+  }
+
+  @Override public int size() {
+    return this.row.length;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/intf/RowIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/intf/RowIntf.java b/core/src/main/java/org/carbondata/scan/filter/intf/RowIntf.java
new file mode 100644
index 0000000..ddfa1eb
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/intf/RowIntf.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.carbondata.scan.filter.intf;
+
+public interface RowIntf {
+  Object getVal(int index);
+
+  Object[] getValues();
+
+  void setValues(Object[] setValues);
+
+  int size();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
new file mode 100644
index 0000000..37685c8
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
@@ -0,0 +1,51 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver;
+
+import java.util.SortedMap;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.scan.filter.intf.ExpressionType;
+
+public class AndFilterResolverImpl extends LogicalFilterResolverImpl {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = -761688076874662001L;
+
+  public AndFilterResolverImpl(FilterResolverIntf leftEvalutor, FilterResolverIntf rightEvalutor,
+      ExpressionType filterExpressionType) {
+    super(leftEvalutor, rightEvalutor, filterExpressionType);
+  }
+
+  @Override public void getStartKey(SegmentProperties segmentProperties, long[] startKeys,
+      SortedMap<Integer, byte[]> noDicStartKeys) {
+    leftEvalutor.getStartKey(segmentProperties, startKeys, noDicStartKeys);
+    rightEvalutor.getStartKey(segmentProperties, startKeys, noDicStartKeys);
+  }
+
+  @Override public void getEndKey(SegmentProperties segmentProperties,
+      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
+      SortedMap<Integer, byte[]> noDicEndKeys) {
+    leftEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
+    rightEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
new file mode 100644
index 0000000..1fb595a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@ -0,0 +1,239 @@
+/*
+ * 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.carbondata.scan.filter.resolver;
+
+import java.util.List;
+import java.util.SortedMap;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.conditional.BinaryConditionalExpression;
+import org.carbondata.scan.expression.conditional.ConditionalExpression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.visitor.FilterInfoTypeVisitorFactory;
+
+public class ConditionalFilterResolverImpl implements FilterResolverIntf {
+
+  private static final long serialVersionUID = 1838955268462201691L;
+  protected Expression exp;
+  protected boolean isExpressionResolve;
+  protected boolean isIncludeFilter;
+  private DimColumnResolvedFilterInfo dimColResolvedFilterInfo;
+
+  public ConditionalFilterResolverImpl(Expression exp, boolean isExpressionResolve,
+      boolean isIncludeFilter) {
+    this.exp = exp;
+    this.isExpressionResolve = isExpressionResolve;
+    this.isIncludeFilter = isIncludeFilter;
+    this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo();
+  }
+
+  /**
+   * This API will resolve the filter expression and generates the
+   * dictionaries for executing/evaluating the filter expressions in the
+   * executer layer.
+   *
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
+      throws FilterUnsupportedException {
+    FilterResolverMetadata metadata = new FilterResolverMetadata();
+    metadata.setTableIdentifier(absoluteTableIdentifier);
+    if ((!isExpressionResolve) && exp instanceof BinaryConditionalExpression) {
+      BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression) exp;
+      Expression leftExp = binaryConditionalExpression.getLeft();
+      Expression rightExp = binaryConditionalExpression.getRight();
+      if (leftExp instanceof ColumnExpression) {
+        ColumnExpression columnExpression = (ColumnExpression) leftExp;
+        metadata.setColumnExpression(columnExpression);
+        metadata.setExpression(rightExp);
+        metadata.setIncludeFilter(isIncludeFilter);
+        // If imei=imei comes in filter condition then we need to
+        // skip processing of right expression.
+        // This flow has reached here assuming that this is a single
+        // column expression.
+        // we need to check if the other expression contains column
+        // expression or not in depth.
+        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)||
+            FilterUtil.isExpressionNeedsToResolved(rightExp,isIncludeFilter) &&
+            columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)){
+          isExpressionResolve = true;
+        } else {
+          //Visitor pattern is been used in this scenario inorder to populate the
+          // dimColResolvedFilterInfo
+          //visitable object with filter member values based on the visitor type, currently there
+          //3 types of visitors custom,direct and no dictionary, all types of visitor populate
+          //the visitable instance as per its buisness logic which is different for all the
+          // visitors.
+          dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
+              FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
+              metadata);
+        }
+      } else if (rightExp instanceof ColumnExpression) {
+        ColumnExpression columnExpression = (ColumnExpression) rightExp;
+        metadata.setColumnExpression(columnExpression);
+        metadata.setExpression(leftExp);
+        metadata.setIncludeFilter(isIncludeFilter);
+        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
+          isExpressionResolve = true;
+        } else {
+          // if imei=imei comes in filter condition then we need to
+          // skip processing of right expression.
+          // This flow has reached here assuming that this is a single
+          // column expression.
+          // we need to check if the other expression contains column
+          // expression or not in depth.
+          if (FilterUtil.checkIfExpressionContainsColumn(leftExp)) {
+            isExpressionResolve = true;
+          } else {
+
+            dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
+                FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
+                metadata);
+
+          }
+        }
+      } else {
+        isExpressionResolve = true;
+      }
+    }
+    if (isExpressionResolve && exp instanceof ConditionalExpression) {
+      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
+      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
+      metadata.setColumnExpression(columnList.get(0));
+      metadata.setExpression(exp);
+      metadata.setIncludeFilter(isIncludeFilter);
+      if (!columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) || columnList.get(0)
+          .getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+        dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
+            FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnList.get(0)), metadata);
+
+      } else if (columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) && !(
+          columnList.get(0).getDimension().getDataType()
+              == org.carbondata.core.carbon.metadata.datatype.DataType.STRUCT
+              || columnList.get(0).getDimension().getDataType()
+              == org.carbondata.core.carbon.metadata.datatype.DataType.ARRAY)) {
+        dimColResolvedFilterInfo.setFilterValues(FilterUtil
+            .getFilterListForAllValues(absoluteTableIdentifier, exp, columnList.get(0),
+                isIncludeFilter));
+
+        dimColResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
+        dimColResolvedFilterInfo.setDimension(columnList.get(0).getDimension());
+      }
+    }
+
+  }
+
+  /**
+   * Left node will not be presentin this scenario
+   *
+   * @return left node of type FilterResolverIntf instance
+   */
+  public FilterResolverIntf getLeft() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  /**
+   * Right node will not be presentin this scenario
+   *
+   * @return left node of type FilterResolverIntf instance
+   */
+  @Override public FilterResolverIntf getRight() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which consists
+   * the mapping of the respective dimension and its surrogates involved in
+   * filter expression.
+   *
+   * @return DimColumnResolvedFilterInfo
+   */
+  public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
+    return dimColResolvedFilterInfo;
+  }
+
+  /**
+   * method will calculates the start key based on the filter surrogates
+   */
+  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
+      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
+    if (null == dimColResolvedFilterInfo.getStarIndexKey()) {
+      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
+          setOfStartKeyByteArray);
+    }
+  }
+
+  /**
+   * method will get the start key based on the filter surrogates
+   *
+   * @return end IndexKey
+   */
+  @Override public void getEndKey(SegmentProperties segmentProperties,
+      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
+      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
+    if (null == dimColResolvedFilterInfo.getEndIndexKey()) {
+      try {
+        FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
+            absoluteTableIdentifier, endKeys, segmentProperties);
+        FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
+            setOfEndKeyByteArray);
+      } catch (QueryExecutionException e) {
+        // TODO Auto-generated catch block
+        e.printStackTrace();
+      }
+    }
+  }
+
+  /**
+   * Method will return the executer type for particular conditional resolver
+   * basically two types of executers will be formed for the conditional query.
+   *
+   * @return the filter executer type
+   */
+  @Override public FilterExecuterType getFilterExecuterType() {
+    switch (exp.getFilterExpressionType()) {
+      case NOT_EQUALS:
+      case NOT_IN:
+        return FilterExecuterType.EXCLUDE;
+
+      default:
+        return FilterExecuterType.INCLUDE;
+    }
+
+  }
+
+  @Override public Expression getFilterExpression() {
+    // TODO Auto-generated method stub
+    return exp;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java b/core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
new file mode 100644
index 0000000..620b1ba
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
@@ -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.carbondata.scan.filter.resolver;
+
+import java.io.Serializable;
+import java.util.SortedMap;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public interface FilterResolverIntf extends Serializable {
+
+  /**
+   * This API will resolve the filter expression and generates the
+   * dictionaries for executing/evaluating the filter expressions in the
+   * executer layer.
+   *
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) throws FilterUnsupportedException;
+
+  /**
+   * This API will provide the left column filter expression
+   * inorder to resolve the left expression filter.
+   *
+   * @return FilterResolverIntf
+   */
+  FilterResolverIntf getLeft();
+
+  /**
+   * API will provide the right column filter expression inorder to resolve
+   * the right expression filter.
+   *
+   * @return FilterResolverIntf
+   */
+  FilterResolverIntf getRight();
+
+  /**
+   * API will return the resolved filter instance, this instance will provide
+   * the resolved surrogates based on the applied filter
+   *
+   * @return DimColumnResolvedFilterInfo object
+   */
+  DimColumnResolvedFilterInfo getDimColResolvedFilterInfo();
+
+  /**
+   * API will get the start key based on the filter applied based on the key generator
+   *
+   * @param segmentProperties
+   * @param startKey
+   * @param setOfStartKeyByteArray
+   */
+  void getStartKey(SegmentProperties segmentProperties, long[] startKey,
+      SortedMap<Integer, byte[]> setOfStartKeyByteArray);
+
+  /**
+   * API will read the end key based on the max surrogate of
+   * particular dimension column
+   *
+   * @param setOfEndKeyByteArray
+   * @param endKeys
+   * @return
+   */
+  void getEndKey(SegmentProperties segmentProperties, AbsoluteTableIdentifier tableIdentifier,
+      long[] endKeys, SortedMap<Integer, byte[]> setOfEndKeyByteArray);
+
+  /**
+   * API will return the filter executer type which will be used to evaluate
+   * the resolved filter while query execution
+   *
+   * @return FilterExecuterType.
+   */
+  FilterExecuterType getFilterExecuterType();
+
+  Expression getFilterExpression();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
new file mode 100644
index 0000000..48c43be
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
@@ -0,0 +1,110 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.filter.resolver;
+
+import java.util.SortedMap;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class LogicalFilterResolverImpl implements FilterResolverIntf {
+  /**
+   *
+   */
+  private static final long serialVersionUID = 5734382980564402914L;
+
+  protected FilterResolverIntf leftEvalutor;
+
+  protected FilterResolverIntf rightEvalutor;
+
+  protected ExpressionType filterExpressionType;
+
+  public LogicalFilterResolverImpl(FilterResolverIntf leftEvalutor,
+      FilterResolverIntf rightEvalutor, ExpressionType filterExpressionType) {
+    this.leftEvalutor = leftEvalutor;
+    this.rightEvalutor = rightEvalutor;
+    this.filterExpressionType = filterExpressionType;
+  }
+
+  /**
+   * Logical filter resolver will return the left and right filter expresison
+   * node for filter evaluation, so in this instance no implementation is required.
+   *
+   * @param absoluteTableIdentifier
+   */
+  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+
+  }
+
+  /**
+   * Since its a binary condition expresion the getLeft method will get the left
+   * node of filter expression
+   *
+   * @return FilterResolverIntf.
+   */
+  public FilterResolverIntf getLeft() {
+    return leftEvalutor;
+  }
+
+  /**
+   * Since its a binary condition expresion the getRight method will get the left
+   * node of filter expression
+   *
+   * @return FilterResolverIntf.
+   */
+  public FilterResolverIntf getRight() {
+    return rightEvalutor;
+  }
+
+  @Override public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
+    return null;
+  }
+
+  @Override public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
+      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
+
+  }
+
+  @Override public void getEndKey(SegmentProperties segmentProperties,
+      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
+      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
+  }
+
+  @Override public FilterExecuterType getFilterExecuterType() {
+    switch (filterExpressionType) {
+      case OR:
+        return FilterExecuterType.OR;
+      case AND:
+        return FilterExecuterType.AND;
+
+      default:
+        return null;
+    }
+  }
+
+  @Override public Expression getFilterExpression() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
new file mode 100644
index 0000000..e91e6de
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
@@ -0,0 +1,211 @@
+/*
+ * 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.carbondata.scan.filter.resolver;
+
+import java.util.List;
+import java.util.SortedMap;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.conditional.BinaryConditionalExpression;
+import org.carbondata.scan.expression.conditional.ConditionalExpression;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class RestructureFilterResolverImpl implements FilterResolverIntf {
+  /**
+   *
+   */
+  private static final long serialVersionUID = -5399656036192814524L;
+
+  protected DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo;
+
+  private Expression exp;
+
+  private String defaultValue;
+
+  private int surrogate;
+
+  private boolean isExpressionResolve;
+
+  private boolean isIncludeFilter;
+
+  public RestructureFilterResolverImpl(Expression exp, String defaultValue, int surrogate,
+      boolean isExpressionResolve, boolean isIncludeFilter) {
+    dimColumnResolvedFilterInfo = new DimColumnResolvedFilterInfo();
+    this.exp = exp;
+    this.defaultValue = defaultValue;
+    this.surrogate = surrogate;
+    this.isExpressionResolve = isExpressionResolve;
+    this.isIncludeFilter = isIncludeFilter;
+  }
+
+  /**
+   * Method will resolve the filters and it will replace the newly added dimension with default
+   * value
+   *
+   * @param absoluteTableIdentifier
+   */
+  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+
+    DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo = new DimColumnResolvedFilterInfo();
+    if (!this.isExpressionResolve && exp instanceof BinaryConditionalExpression) {
+      BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression) exp;
+      Expression left = binaryConditionalExpression.getLeft();
+      Expression right = binaryConditionalExpression.getRight();
+      if (left instanceof ColumnExpression) {
+        ColumnExpression columnExpression = (ColumnExpression) left;
+        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
+          isExpressionResolve = true;
+        } else {
+          // If imei=imei comes in filter condition then we need to
+          // skip processing of right expression.
+          // This flow has reached here assuming that this is a single
+          // column expression.
+          // we need to check if the other expression contains column
+          // expression or not in depth.
+          if (FilterUtil.checkIfExpressionContainsColumn(right)) {
+            isExpressionResolve = true;
+          } else {
+            dimColumnResolvedFilterInfo
+                .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
+            // dimColumnResolvedFilterInfo
+            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
+            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
+            // .getOrdinal()]);
+            dimColumnResolvedFilterInfo.setFilterValues(
+                FilterUtil.getFilterListForRS(right, columnExpression, defaultValue, surrogate));
+          }
+        }
+      } else if (right instanceof ColumnExpression) {
+        ColumnExpression columnExpression = (ColumnExpression) right;
+        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
+          isExpressionResolve = true;
+        } else {
+
+          // If imei=imei comes in filter condition then we need to
+          // skip processing of right expression.
+          // This flow has reached here assuming that this is a single
+          // column expression.
+          // we need to check if the other expression contains column
+          // expression or not in depth.
+          if (checkIfExpressionContainsColumn(left)) {
+            isExpressionResolve = true;
+          } else {
+            dimColumnResolvedFilterInfo
+                .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
+            // dimColumnResolvedFilterInfo
+            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
+            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
+            // .getOrdinal()]);
+            dimColumnResolvedFilterInfo.setFilterValues(
+                FilterUtil.getFilterListForRS(left, columnExpression, defaultValue, surrogate));
+          }
+        }
+      }
+    }
+    if (this.isExpressionResolve && exp instanceof ConditionalExpression) {
+      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
+      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
+      dimColumnResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
+      dimColumnResolvedFilterInfo.setFilterValues(FilterUtil
+          .getFilterListForAllMembersRS(exp, columnList.get(0), defaultValue, surrogate,
+              isIncludeFilter));
+    }
+
+  }
+
+  /**
+   * This method will check if a given expression contains a column expression recursively.
+   *
+   * @return boolean
+   */
+  private boolean checkIfExpressionContainsColumn(Expression expression) {
+    if (expression instanceof ColumnExpression) {
+      return true;
+    }
+    for (Expression child : expression.getChildren()) {
+      if (checkIfExpressionContainsColumn(child)) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  @Override public FilterResolverIntf getLeft() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override public FilterResolverIntf getRight() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which consists
+   * the mapping of the respective dimension and its surrogates involved in
+   * filter expression.
+   *
+   * @return DimColumnResolvedFilterInfo
+   */
+  public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
+    return dimColumnResolvedFilterInfo;
+  }
+
+  /**
+   * For restructure resolver no implementation is required for getting
+   * the start key since it already has default values
+   */
+  @Override public void getStartKey(SegmentProperties segmentProperties, long[] startKeys,
+      SortedMap<Integer, byte[]> noDicStartKeys) {
+
+  }
+
+  /**
+   * For restructure resolver no implementation is required for getting
+   * the end  key since it already has default values
+   *
+   * @return IndexKey.
+   */
+  @Override public void getEndKey(SegmentProperties segmentProperties,
+      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
+      SortedMap<Integer, byte[]> noDicEndKeys) {
+  }
+
+  /**
+   * Method will get the executer type inorder to create filter executer tree
+   *
+   * @return FilterExecuterType
+   */
+  @Override public FilterExecuterType getFilterExecuterType() {
+    return FilterExecuterType.RESTRUCTURE;
+  }
+
+  @Override public Expression getFilterExpression() {
+    // TODO Auto-generated method stub
+    return exp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java
new file mode 100644
index 0000000..7a26c12
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelFilterResolverImpl.java
@@ -0,0 +1,141 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.filter.resolver;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.conditional.ConditionalExpression;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+
+public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl {
+
+  private static final long serialVersionUID = 176122729713729929L;
+  protected boolean isExpressionResolve;
+  protected boolean isIncludeFilter;
+
+  private List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
+  private List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
+  private AbsoluteTableIdentifier tableIdentifier;
+
+  public RowLevelFilterResolverImpl(Expression exp, boolean isExpressionResolve,
+      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
+    super(exp, isExpressionResolve, isIncludeFilter);
+    dimColEvaluatorInfoList =
+        new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(
+        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.tableIdentifier = tableIdentifier;
+  }
+
+  /**
+   * Method which will resolve the filter expression by converting the filter member
+   * to its assigned dictionary values.
+   */
+  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+    DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = null;
+    MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = null;
+    int index = 0;
+    if (exp instanceof ConditionalExpression) {
+      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
+      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
+      for (ColumnExpression columnExpression : columnList) {
+        if (columnExpression.isDimension()) {
+          dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
+          dimColumnEvaluatorInfo.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
+          dimColumnEvaluatorInfo.setRowIndex(index++);
+          dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
+          dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
+          dimColEvaluatorInfoList.add(dimColumnEvaluatorInfo);
+        } else {
+          msrColumnEvalutorInfo = new MeasureColumnResolvedFilterInfo();
+          msrColumnEvalutorInfo.setRowIndex(index++);
+          msrColumnEvalutorInfo.setAggregator(
+              ((CarbonMeasure) columnExpression.getCarbonColumn()).getAggregateFunction());
+          msrColumnEvalutorInfo
+              .setColumnIndex(((CarbonMeasure) columnExpression.getCarbonColumn()).getOrdinal());
+          msrColumnEvalutorInfo.setType(columnExpression.getCarbonColumn().getDataType());
+          msrColEvalutorInfoList.add(msrColumnEvalutorInfo);
+        }
+      }
+    }
+  }
+
+  /**
+   * This method will provide the executer type to the callee inorder to identify
+   * the executer type for the filter resolution, Row level filter executer is a
+   * special executer since it get all the rows of the specified filter dimension
+   * and will be send to the spark for processing
+   */
+  @Override public FilterExecuterType getFilterExecuterType() {
+    return FilterExecuterType.ROWLEVEL;
+  }
+
+  /**
+   * Method will the read filter expression corresponding to the resolver.
+   * This method is required in row level executer inorder to evaluate the filter
+   * expression against spark, as mentioned above row level is a special type
+   * filter resolver.
+   *
+   * @return Expression
+   */
+  public Expression getFilterExpresion() {
+    return exp;
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which consists
+   * the mapping of the respective dimension and its surrogates involved in
+   * filter expression.
+   *
+   * @return DimColumnResolvedFilterInfo
+   */
+  public List<DimColumnResolvedFilterInfo> getDimColEvaluatorInfoList() {
+    return dimColEvaluatorInfoList;
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which containts
+   * measure level details.
+   *
+   * @return MeasureColumnResolvedFilterInfo
+   */
+  public List<MeasureColumnResolvedFilterInfo> getMsrColEvalutorInfoList() {
+    return msrColEvalutorInfoList;
+  }
+
+  /**
+   * Method will return table information which will be required for retrieving
+   * dictionary cache inorder to read all the members of respective dimension.
+   *
+   * @return AbsoluteTableIdentifier
+   */
+  public AbsoluteTableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+}


[16/56] [abbrv] incubator-carbondata git commit: Refactored core package and fixed all testcases (#684)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/impl/MapBasedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/impl/MapBasedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/impl/MapBasedResult.java
deleted file mode 100644
index 42c3027..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/impl/MapBasedResult.java
+++ /dev/null
@@ -1,141 +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.carbondata.query.carbon.result.impl;
-
-import java.util.HashMap;
-import java.util.Iterator;
-import java.util.Map;
-import java.util.Map.Entry;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * To store aggregated result
- */
-public class MapBasedResult implements Result<Map<ByteArrayWrapper, MeasureAggregator[]>> {
-  /**
-   * iterator over result
-   */
-  private Iterator<Entry<ByteArrayWrapper, MeasureAggregator[]>> resultIterator;
-
-  /**
-   * result entry
-   */
-  private Entry<ByteArrayWrapper, MeasureAggregator[]> resultEntry;
-
-  /**
-   * scanned result
-   */
-  private Map<ByteArrayWrapper, MeasureAggregator[]> scannerResult;
-
-  /**
-   * total number of result
-   */
-  private int resulSize;
-
-  public MapBasedResult() {
-    scannerResult = new HashMap<ByteArrayWrapper, MeasureAggregator[]>(
-        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    this.resultIterator = scannerResult.entrySet().iterator();
-  }
-
-  /**
-   * @return the key
-   */
-  @Override public ByteArrayWrapper getKey() {
-    resultEntry = this.resultIterator.next();
-    return resultEntry.getKey();
-  }
-
-  /**
-   * return the value
-   */
-  @Override public MeasureAggregator[] getValue() {
-    return resultEntry.getValue();
-  }
-
-  /**
-   * Method to check more result is present
-   * or not
-   */
-  @Override public boolean hasNext() {
-    return this.resultIterator.hasNext();
-  }
-
-  /***
-   * below method will be used to merge the
-   * scanned result
-   *
-   * @param otherResult return to be merged
-   */
-  @Override public void addScannedResult(Map<ByteArrayWrapper, MeasureAggregator[]> scannerResult) {
-    this.scannerResult = scannerResult;
-    resulSize = scannerResult.size();
-    this.resultIterator = scannerResult.entrySet().iterator();
-  }
-
-  /***
-   * below method will be used to merge the
-   * scanned result, in case of map based the
-   * result we need to aggregate the result
-   *
-   * @param otherResult return to be merged
-   */
-  @Override public void merge(Result<Map<ByteArrayWrapper, MeasureAggregator[]>> result) {
-    ByteArrayWrapper key = null;
-    MeasureAggregator[] value = null;
-    Map<ByteArrayWrapper, MeasureAggregator[]> otherResult = result.getResult();
-    if (otherResult != null) {
-      while (resultIterator.hasNext()) {
-        Entry<ByteArrayWrapper, MeasureAggregator[]> entry = resultIterator.next();
-        key = entry.getKey();
-        value = entry.getValue();
-        MeasureAggregator[] agg = otherResult.get(key);
-        if (agg != null) {
-          for (int j = 0; j < agg.length; j++) {
-            agg[j].merge(value[j]);
-          }
-        } else {
-          otherResult.put(key, value);
-        }
-      }
-      resulSize = otherResult.size();
-      this.resultIterator = otherResult.entrySet().iterator();
-      this.scannerResult = otherResult;
-    }
-  }
-
-  /**
-   * Return the size of the result
-   */
-  @Override public int size() {
-    return resulSize;
-  }
-
-  /**
-   * @return the complete result
-   */
-  @Override public Map<ByteArrayWrapper, MeasureAggregator[]> getResult() {
-    return this.scannerResult;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
index bd56ec2..09fa50c 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
@@ -39,7 +39,7 @@ import org.carbondata.query.carbon.model.QueryModel;
  * executing that query are returning a iterator over block and every time next
  * call will come it will execute the block and return the result
  */
-public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterator<E> {
+public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
 
   /**
    * LOGGER.
@@ -75,7 +75,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
   /**
    * current counter to check how blocklet has been executed
    */
-  private long currentCounter;
+  protected long currentCounter;
 
   /**
    * keep the track of number of blocklet of a block has been executed
@@ -138,7 +138,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
     return currentCounter < totalNumberOfNode;
   }
 
-  protected void updateSliceIndexToBeExecuted() {
+  protected int updateSliceIndexToBeExecuted() {
     Arrays.fill(blockIndexToBeExecuted, -1);
     int currentSliceIndex = 0;
     int i = 0;
@@ -154,7 +154,7 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
         break;
       }
     }
-    currentCounter += i;
+    return i;
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java
new file mode 100644
index 0000000..826f816
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java
@@ -0,0 +1,75 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.carbon.result.iterator;
+
+import java.util.List;
+
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
+import org.carbondata.query.carbon.model.QueryModel;
+import org.carbondata.query.carbon.result.BatchResult;
+import org.carbondata.query.carbon.result.ListBasedResultWrapper;
+import org.carbondata.query.carbon.result.Result;
+import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
+import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPreparatorImpl;
+
+/**
+ * Iterator over chunk result
+ */
+public class ChunkBasedDetailResultIterator extends CarbonIterator<BatchResult> {
+
+  /**
+   * query result prepartor which will be used to create a query result
+   */
+  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
+
+  /**
+   * iterator over result
+   */
+  private CarbonIterator<Result> queryResultIterator;
+
+  public ChunkBasedDetailResultIterator(CarbonIterator<Result> queryResultIterator,
+      QueryExecutorProperties executerProperties, QueryModel queryModel) {
+    this.queryResultIterator = queryResultIterator;
+    this.queryResultPreparator =
+        new DetailQueryResultPreparatorImpl(executerProperties, queryModel);
+
+  }
+
+  /**
+   * Returns {@code true} if the iteration has more elements. (In other words,
+   * returns {@code true}
+   *
+   * @return {@code true} if the iteration has more elements
+   */
+  @Override public boolean hasNext() {
+    return queryResultIterator.hasNext();
+  }
+
+  /**
+   * Returns the next element in the iteration.
+   *
+   * @return the next element in the iteration
+   */
+  @Override public BatchResult next() {
+    return queryResultPreparator.prepareQueryResult(queryResultIterator.next());
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedResultIterator.java
deleted file mode 100644
index 71b311f..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedResultIterator.java
+++ /dev/null
@@ -1,71 +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.carbondata.query.carbon.result.iterator;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.result.preparator.impl.QueryResultPreparatorImpl;
-
-/**
- * Iterator over chunk result
- */
-public class ChunkBasedResultIterator extends CarbonIterator<BatchResult> {
-
-  /**
-   * query result prepartor which will be used to create a query result
-   */
-  private QueryResultPreparator<BatchResult> queryResultPreparator;
-
-  /**
-   * iterator over result
-   */
-  private CarbonIterator<Result> queryResultIterator;
-
-  public ChunkBasedResultIterator(CarbonIterator<Result> queryResultIterator,
-      QueryExecutorProperties executerProperties, QueryModel queryModel) {
-    this.queryResultIterator = queryResultIterator;
-    this.queryResultPreparator = new QueryResultPreparatorImpl(executerProperties, queryModel);
-
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements. (In other words,
-   * returns {@code true}
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return queryResultIterator.hasNext();
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public BatchResult next() {
-    return queryResultPreparator.prepareQueryResult(queryResultIterator.next());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRawRowIterartor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRawRowIterartor.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRawRowIterartor.java
deleted file mode 100644
index ea4d65c..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRawRowIterartor.java
+++ /dev/null
@@ -1,56 +0,0 @@
-package org.carbondata.query.carbon.result.iterator;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.result.BatchRawResult;
-
-public class ChunkRawRowIterartor extends CarbonIterator<Object[]> {
-
-  /**
-   * iterator over chunk result
-   */
-  private CarbonIterator<BatchRawResult> iterator;
-
-  /**
-   * currect chunk
-   */
-  private BatchRawResult currentchunk;
-
-  public ChunkRawRowIterartor(CarbonIterator<BatchRawResult> iterator) {
-    this.iterator = iterator;
-    if (iterator.hasNext()) {
-      currentchunk = iterator.next();
-    }
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements. (In other words,
-   * returns {@code true} if {@link #next} would return an element rather than
-   * throwing an exception.)
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    if (null != currentchunk) {
-      if ((currentchunk.hasNext())) {
-        return true;
-      } else if (!currentchunk.hasNext()) {
-        while (iterator.hasNext()) {
-          currentchunk = iterator.next();
-          if (currentchunk != null && currentchunk.hasNext()) {
-            return true;
-          }
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Object[] next() {
-    return currentchunk.next();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java
index 6ba54cd..3db3404 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java
@@ -21,12 +21,11 @@ package org.carbondata.query.carbon.result.iterator;
 
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.RowResult;
 
 /**
  * Iterator over row result
  */
-public class ChunkRowIterator extends CarbonIterator<RowResult> {
+public class ChunkRowIterator extends CarbonIterator<Object[]> {
 
   /**
    * iterator over chunk result
@@ -73,7 +72,7 @@ public class ChunkRowIterator extends CarbonIterator<RowResult> {
    *
    * @return the next element in the iteration
    */
-  @Override public RowResult next() {
+  @Override public Object[] next() {
     return currentchunk.next();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
index f07eb20..3641e75 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
@@ -29,16 +29,17 @@ import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
 import org.carbondata.query.carbon.result.BatchResult;
+import org.carbondata.query.carbon.result.ListBasedResultWrapper;
 import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.result.preparator.impl.QueryResultPreparatorImpl;
+import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPreparatorImpl;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
  * executing that query are returning a iterator over block and every time next
  * call will come it will execute the block and return the result
  */
-public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator<BatchResult> {
+public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator {
 
   /**
    * LOGGER.
@@ -49,17 +50,18 @@ public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator
   /**
    * to prepare the result
    */
-  private QueryResultPreparator<BatchResult> queryResultPreparator;
+  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
 
   public DetailQueryResultIterator(List<BlockExecutionInfo> infos,
       QueryExecutorProperties executerProperties, QueryModel queryModel,
       InternalQueryExecutor queryExecutor) {
     super(infos, executerProperties, queryModel, queryExecutor);
-    this.queryResultPreparator = new QueryResultPreparatorImpl(executerProperties, queryModel);
+    this.queryResultPreparator =
+        new DetailQueryResultPreparatorImpl(executerProperties, queryModel);
   }
 
   @Override public BatchResult next() {
-    updateSliceIndexToBeExecuted();
+    currentCounter += updateSliceIndexToBeExecuted();
     CarbonIterator<Result> result = null;
     try {
       result = executor.executeQuery(blockExecutionInfos, blockIndexToBeExecuted);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
index 4f9dbe2..2b14793 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
@@ -19,6 +19,10 @@
 package org.carbondata.query.carbon.result.iterator;
 
 import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
@@ -26,7 +30,8 @@ import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchRawResult;
+import org.carbondata.query.carbon.result.BatchResult;
+import org.carbondata.query.carbon.result.ListBasedResultWrapper;
 import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
 import org.carbondata.query.carbon.result.preparator.impl.RawQueryResultPreparatorImpl;
@@ -36,10 +41,13 @@ import org.carbondata.query.carbon.result.preparator.impl.RawQueryResultPreparat
  * executing that query are returning a iterator over block and every time next
  * call will come it will execute the block and return the result
  */
-public class DetailRawQueryResultIterator
-    extends AbstractDetailQueryResultIterator<BatchRawResult> {
+public class DetailRawQueryResultIterator extends AbstractDetailQueryResultIterator {
 
-  private QueryResultPreparator<BatchRawResult> queryResultPreparator;
+  private ExecutorService execService = Executors.newFixedThreadPool(1);
+
+  private Future<ResultInfo> future;
+
+  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
 
   public DetailRawQueryResultIterator(List<BlockExecutionInfo> infos,
       QueryExecutorProperties executerProperties, QueryModel queryModel,
@@ -48,26 +56,63 @@ public class DetailRawQueryResultIterator
     this.queryResultPreparator = new RawQueryResultPreparatorImpl(executerProperties, queryModel);
   }
 
-  @Override public BatchRawResult next() {
-    updateSliceIndexToBeExecuted();
-    CarbonIterator<Result> result = null;
-    try {
-      result = executor.executeQuery(blockExecutionInfos, blockIndexToBeExecuted);
-    } catch (QueryExecutionException ex) {
-      throw new RuntimeException(ex.getCause());
+  @Override public BatchResult next() {
+    BatchResult result;
+    if (future == null) {
+      future = execute();
     }
-    for (int i = 0; i < blockIndexToBeExecuted.length; i++) {
-      if (blockIndexToBeExecuted[i] != -1) {
-        blockExecutionInfos.get(blockIndexToBeExecuted[i]).setFirstDataBlock(
-            blockExecutionInfos.get(blockIndexToBeExecuted[i]).getFirstDataBlock()
-                .getNextDataRefNode());
-      }
+    ResultInfo resultFromFuture = getResultFromFuture(future);
+    result = resultFromFuture.result;
+    currentCounter += resultFromFuture.counter;
+    if (hasNext()) {
+      future = execute();
     }
-    if (null != result) {
-      Result next = result.next();
-      return queryResultPreparator.prepareQueryResult(next);
-    } else {
-      return queryResultPreparator.prepareQueryResult(null);
+    return result;
+  }
+
+  private ResultInfo getResultFromFuture(Future<ResultInfo> future) {
+    try {
+      return future.get();
+    } catch (Exception e) {
+      e.printStackTrace();
     }
+    return new ResultInfo();
+  }
+
+  private Future<ResultInfo> execute() {
+    return execService.submit(new Callable<ResultInfo>() {
+      @Override public ResultInfo call() {
+        CarbonIterator<Result> result = null;
+        int counter =  updateSliceIndexToBeExecuted();
+        try {
+          result = executor.executeQuery(blockExecutionInfos, blockIndexToBeExecuted);
+        } catch (QueryExecutionException ex) {
+          throw new RuntimeException(ex.getCause());
+        }
+        for (int i = 0; i < blockIndexToBeExecuted.length; i++) {
+          if (blockIndexToBeExecuted[i] != -1) {
+            blockExecutionInfos.get(blockIndexToBeExecuted[i]).setFirstDataBlock(
+                blockExecutionInfos.get(blockIndexToBeExecuted[i]).getFirstDataBlock()
+                    .getNextDataRefNode());
+          }
+        }
+        BatchResult batchResult;
+        if (null != result) {
+          Result next = result.next();
+          batchResult = queryResultPreparator.prepareQueryResult(next);
+        } else {
+          batchResult = queryResultPreparator.prepareQueryResult(null);
+        }
+        ResultInfo resultInfo = new ResultInfo();
+        resultInfo.counter = counter;
+        resultInfo.result = batchResult;
+        return resultInfo;
+      }
+    });
+  }
+
+  private static class ResultInfo {
+    private int counter;
+    private BatchResult result;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
index 431e163..fbf3074 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
@@ -1,9 +1,10 @@
 package org.carbondata.query.carbon.result.preparator;
 
+import org.carbondata.query.carbon.result.BatchResult;
 import org.carbondata.query.carbon.result.Result;
 
-public interface QueryResultPreparator<E> {
+public interface QueryResultPreparator<K, V> {
 
-  public E prepareQueryResult(Result scannedResult);
+  public BatchResult prepareQueryResult(Result<K, V> scannedResult);
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
index 1890baf..ad5b4c5 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
@@ -1,17 +1,19 @@
 package org.carbondata.query.carbon.result.preparator.impl;
 
-import java.util.ArrayList;
 import java.util.List;
 
-import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
+import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QueryModel;
 import org.carbondata.query.carbon.result.BatchResult;
 import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.scanner.impl.CarbonKey;
-import org.carbondata.query.scanner.impl.CarbonValue;
+import org.carbondata.query.carbon.util.DataTypeUtil;
 
-public abstract class AbstractQueryResultPreparator<E> implements QueryResultPreparator<E> {
+public abstract class AbstractQueryResultPreparator<K, V> implements QueryResultPreparator<K, V> {
 
   /**
    * query properties
@@ -29,13 +31,35 @@ public abstract class AbstractQueryResultPreparator<E> implements QueryResultPre
     this.queryModel = queryModel;
   }
 
-  protected void fillMeasureValueForAggGroupByQuery(QueryModel queryModel,
-      Object[][] surrogateResult, int dimensionCount, int columnIndex, MeasureAggregator[] v) {
-    int msrCount = queryModel.getQueryMeasures().size();
-    for (int i = 0; i < msrCount; i++) {
-      v[queryExecuterProperties.measureStartIndex + i] =
-          ((MeasureAggregator) surrogateResult[dimensionCount
-              + queryExecuterProperties.measureStartIndex + i][columnIndex]);
+  protected void fillDimensionData(Object[][] convertedResult, List<QueryDimension> queryDimensions,
+      int dimensionCount, Object[] row, int rowIndex) {
+    QueryDimension queryDimension;
+    for (int i = 0; i < dimensionCount; i++) {
+      queryDimension = queryDimensions.get(i);
+      if (!CarbonUtil
+          .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)) {
+        row[queryDimension.getQueryOrder()] = convertedResult[i][rowIndex];
+      } else if (CarbonUtil
+          .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DIRECT_DICTIONARY)) {
+        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+            .getDirectDictionaryGenerator(queryDimension.getDimension().getDataType());
+        row[queryDimension.getQueryOrder()] = directDictionaryGenerator
+            .getValueFromSurrogate((Integer) convertedResult[i][rowIndex]);
+      } else {
+        if (queryExecuterProperties.sortDimIndexes[i] == 1) {
+          row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
+              queryExecuterProperties.columnToDictionayMapping
+                  .get(queryDimension.getDimension().getColumnId())
+                  .getDictionaryValueFromSortedIndex((Integer) convertedResult[i][rowIndex]),
+              queryDimension.getDimension().getDataType());
+        } else {
+          row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
+              queryExecuterProperties.columnToDictionayMapping
+                  .get(queryDimension.getDimension().getColumnId())
+                  .getDictionaryValueForKey((Integer) convertedResult[i][rowIndex]),
+              queryDimension.getDimension().getDataType());
+        }
+      }
     }
   }
 
@@ -54,18 +78,9 @@ public abstract class AbstractQueryResultPreparator<E> implements QueryResultPre
   }
 
   protected BatchResult getEmptyChunkResult(int size) {
-    List<CarbonKey> keys = new ArrayList<CarbonKey>(size);
-    List<CarbonValue> values = new ArrayList<CarbonValue>(size);
-    Object[] row = new Object[1];
-    for (int i = 0; i < size; i++)
-
-    {
-      values.add(new CarbonValue(new MeasureAggregator[0]));
-      keys.add(new CarbonKey(row));
-    }
+    Object[][] row = new Object[size][1];
     BatchResult chunkResult = new BatchResult();
-    chunkResult.setKeys(keys);
-    chunkResult.setValues(values);
+    chunkResult.setRows(row);
     return chunkResult;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
new file mode 100644
index 0000000..712894a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
@@ -0,0 +1,139 @@
+/*
+ * 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.carbondata.query.carbon.result.preparator.impl;
+
+import java.nio.charset.Charset;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
+import org.carbondata.query.carbon.model.QueryDimension;
+import org.carbondata.query.carbon.model.QueryMeasure;
+import org.carbondata.query.carbon.model.QueryModel;
+import org.carbondata.query.carbon.result.BatchResult;
+import org.carbondata.query.carbon.result.ListBasedResultWrapper;
+import org.carbondata.query.carbon.result.Result;
+import org.carbondata.query.carbon.util.DataTypeUtil;
+import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
+
+/**
+ * Below class will be used to get the result by converting to actual data
+ * Actual data conversion can be converting the surrogate key to actual data
+ *
+ * @TODO there are many things in class which is very confusing, need to check
+ * why it was handled like that and how we can handle that in a better
+ * way.Need to revisit this class. IF aggregation is push down to spark
+ * layer and if we can process the data in byte array format then this
+ * class wont be useful so in future we can delete this class.
+ * @TODO need to expose one interface which will return the result based on required type
+ * for example its implementation case return converted result or directly result with out
+ * converting to actual value
+ */
+public class DetailQueryResultPreparatorImpl
+    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, Object> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DetailQueryResultPreparatorImpl.class.getName());
+
+  public DetailQueryResultPreparatorImpl(QueryExecutorProperties executerProperties,
+      QueryModel queryModel) {
+    super(executerProperties, queryModel);
+  }
+
+  @Override public BatchResult prepareQueryResult(
+      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
+    if ((null == scannedResult || scannedResult.size() < 1)) {
+      return new BatchResult();
+    }
+    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
+    int dimensionCount = queryDimension.size();
+    int totalNumberOfColumn = dimensionCount + queryExecuterProperties.measureDataTypes.length;
+    Object[][] resultData = new Object[scannedResult.size()][totalNumberOfColumn];
+    if (!queryExecuterProperties.isFunctionQuery && totalNumberOfColumn == 0
+        && scannedResult.size() > 0) {
+      return getEmptyChunkResult(scannedResult.size());
+    }
+    int currentRow = 0;
+    long[] surrogateResult = null;
+    int noDictionaryColumnIndex = 0;
+    ByteArrayWrapper key = null;
+    Object[] value = null;
+    while (scannedResult.hasNext()) {
+      key = scannedResult.getKey();
+      value = scannedResult.getValue();
+      if (key != null) {
+        surrogateResult = queryExecuterProperties.keyStructureInfo.getKeyGenerator()
+            .getKeyArray(key.getDictionaryKey(),
+                queryExecuterProperties.keyStructureInfo.getMaskedBytes());
+        for (int i = 0; i < dimensionCount; i++) {
+          if (!CarbonUtil.hasEncoding(queryDimension.get(i).getDimension().getEncoder(),
+              Encoding.DICTIONARY)) {
+            resultData[currentRow][i] = DataTypeUtil.getDataBasedOnDataType(
+                new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++),
+                    Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
+                queryDimension.get(i).getDimension().getDataType());
+          } else {
+            resultData[currentRow][i] =
+                (int) surrogateResult[queryDimension.get(i).getDimension().getKeyOrdinal()];
+          }
+        }
+      }
+      if (value != null) {
+        System.arraycopy(value, 0, resultData[currentRow], dimensionCount,
+            queryExecuterProperties.measureDataTypes.length);
+      }
+      currentRow++;
+      noDictionaryColumnIndex = 0;
+    }
+    if (resultData.length > 0) {
+      resultData = encodeToRows(resultData);
+    }
+    return getResult(queryModel, resultData);
+  }
+
+  private BatchResult getResult(QueryModel queryModel, Object[][] convertedResult) {
+
+    int rowSize = convertedResult[0].length;
+    Object[][] rows = new Object[rowSize][];
+    List<QueryDimension> queryDimensions = queryModel.getQueryDimension();
+    int dimensionCount = queryDimensions.size();
+    int msrCount = queryExecuterProperties.measureDataTypes.length;
+    Object[] row;
+    for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) {
+      row = new Object[dimensionCount + msrCount];
+      fillDimensionData(convertedResult, queryDimensions, dimensionCount, row, rowIndex);
+
+      QueryMeasure msr;
+      for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) {
+        msr = queryModel.getQueryMeasures().get(i);
+        row[msr.getQueryOrder()] = convertedResult[dimensionCount + i][rowIndex];
+      }
+      rows[rowIndex] = row;
+    }
+    LOGGER.info(
+        "###########################################------ Total Number of records" + rowSize);
+    BatchResult chunkResult = new BatchResult();
+    chunkResult.setRows(rows);
+    return chunkResult;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java
deleted file mode 100644
index 5604ecd..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java
+++ /dev/null
@@ -1,297 +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.carbondata.query.carbon.result.preparator.impl;
-
-import java.math.BigDecimal;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.count.CountAggregator;
-import org.carbondata.query.aggregator.impl.distinct.DistinctCountAggregator;
-import org.carbondata.query.aggregator.impl.distinct.DistinctStringCountAggregator;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-import org.carbondata.query.scanner.impl.CarbonKey;
-import org.carbondata.query.scanner.impl.CarbonValue;
-
-/**
- * Below class will be used to get the result by converting to actual data
- * Actual data conversion can be converting the surrogate key to actual data
- *
- * @TODO there are many things in class which is very confusing, need to check
- * why it was handled like that and how we can handle that in a better
- * way.Need to revisit this class. IF aggregation is push down to spark
- * layer and if we can process the data in byte array format then this
- * class wont be useful so in future we can delete this class.
- * @TODO need to expose one interface which will return the result based on required type
- * for example its implementation case return converted result or directly result with out
- * converting to actual value
- */
-public class QueryResultPreparatorImpl extends AbstractQueryResultPreparator<BatchResult> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(QueryResultPreparatorImpl.class.getName());
-
-  public QueryResultPreparatorImpl(QueryExecutorProperties executerProperties,
-      QueryModel queryModel) {
-    super(executerProperties, queryModel);
-  }
-
-  @Override public BatchResult prepareQueryResult(Result scannedResult) {
-    if ((null == scannedResult || scannedResult.size() < 1)) {
-      return new BatchResult();
-    }
-    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
-    int dimensionCount = queryDimension.size();
-    int totalNumberOfColumn = dimensionCount + queryExecuterProperties.measureAggregators.length;
-    Object[][] resultData = new Object[scannedResult.size()][totalNumberOfColumn];
-    if (!queryExecuterProperties.isFunctionQuery && totalNumberOfColumn == 0
-        && scannedResult.size() > 0) {
-      return getEmptyChunkResult(scannedResult.size());
-    }
-    int currentRow = 0;
-    long[] surrogateResult = null;
-    int noDictionaryColumnIndex = 0;
-    ByteArrayWrapper key = null;
-    MeasureAggregator[] value = null;
-    while (scannedResult.hasNext()) {
-      key = scannedResult.getKey();
-      value = scannedResult.getValue();
-      surrogateResult = queryExecuterProperties.keyStructureInfo.getKeyGenerator()
-          .getKeyArray(key.getDictionaryKey(),
-              queryExecuterProperties.keyStructureInfo.getMaskedBytes());
-      for (int i = 0; i < dimensionCount; i++) {
-        if (!CarbonUtil
-            .hasEncoding(queryDimension.get(i).getDimension().getEncoder(), Encoding.DICTIONARY)) {
-          resultData[currentRow][i] = DataTypeUtil.getDataBasedOnDataType(
-              new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++),
-                  Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
-              queryDimension.get(i).getDimension().getDataType());
-        } else {
-          resultData[currentRow][i] =
-              (int) surrogateResult[queryDimension.get(i).getDimension().getKeyOrdinal()];
-        }
-      }
-
-      // @TODO need to check why it was handled like this
-      if (queryExecuterProperties.isFunctionQuery) {
-        if (value[0].toString().contains("Long")) {
-          Long sizeOfListL = value[0].getLongValue();
-          return getEmptyChunkResult(sizeOfListL.intValue());
-        } else if (value[0].toString().contains("Decimal")) {
-          BigDecimal sizeOfListD = value[0].getBigDecimalValue();
-          return getEmptyChunkResult(sizeOfListD.intValue());
-        } else {
-          Double sizeOfList = value[0].getDoubleValue();
-          return getEmptyChunkResult(sizeOfList.intValue());
-        }
-
-      }
-      for (int i = 0; i < queryExecuterProperties.measureAggregators.length; i++) {
-        resultData[currentRow][dimensionCount + i] = value[i];
-      }
-      currentRow++;
-      noDictionaryColumnIndex = 0;
-    }
-    if (resultData.length > 0) {
-      resultData = encodeToRows(resultData);
-    }
-    return getResult(queryModel, resultData);
-  }
-
-  private BatchResult getResult(QueryModel queryModel, Object[][] convertedResult) {
-
-    List<CarbonKey> keys = new ArrayList<CarbonKey>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<CarbonValue> values =
-        new ArrayList<CarbonValue>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    List<QueryDimension> queryDimensions = queryModel.getQueryDimension();
-    int dimensionCount = queryDimensions.size();
-    int msrCount = queryExecuterProperties.measureAggregators.length;
-    Object[][] resultDataA = null;
-    // @TODO no sure why this check is here as in the caller of this method
-    // is returning in case of
-    // function query. Need to confirm with other developer who handled this
-    // scneario
-    if (queryExecuterProperties.isFunctionQuery) {
-      msrCount = 1;
-      resultDataA = new Object[dimensionCount + msrCount][msrCount];
-    } else {
-      resultDataA = new Object[dimensionCount + msrCount][convertedResult[0].length];
-    }
-    Object[] row = null;
-    QueryDimension queryDimension = null;
-    for (int columnIndex = 0; columnIndex < resultDataA[0].length; columnIndex++) {
-      row = new Object[dimensionCount + msrCount];
-      for (int i = 0; i < dimensionCount; i++) {
-        queryDimension = queryDimensions.get(i);
-        if (!CarbonUtil
-            .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)) {
-          row[queryDimension.getQueryOrder()] = convertedResult[i][columnIndex];
-        } else if (CarbonUtil
-            .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DIRECT_DICTIONARY)) {
-          DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-              .getDirectDictionaryGenerator(queryDimension.getDimension().getDataType());
-          row[queryDimension.getQueryOrder()] = directDictionaryGenerator
-              .getValueFromSurrogate((Integer) convertedResult[i][columnIndex]);
-        } else {
-          if (queryExecuterProperties.sortDimIndexes[i] == 1) {
-            row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
-                queryExecuterProperties.columnToDictionayMapping
-                    .get(queryDimension.getDimension().getColumnId())
-                    .getDictionaryValueFromSortedIndex((Integer) convertedResult[i][columnIndex]),
-                queryDimension.getDimension().getDataType());
-          } else {
-            row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
-                queryExecuterProperties.columnToDictionayMapping
-                    .get(queryDimension.getDimension().getColumnId())
-                    .getDictionaryValueForKey((Integer) convertedResult[i][columnIndex]),
-                queryDimension.getDimension().getDataType());
-          }
-        }
-      }
-      MeasureAggregator[] msrAgg =
-          new MeasureAggregator[queryExecuterProperties.measureAggregators.length];
-
-      fillMeasureValueForAggGroupByQuery(queryModel, convertedResult, dimensionCount, columnIndex,
-          msrAgg);
-      fillDimensionAggValue(queryModel, convertedResult, dimensionCount, columnIndex, msrAgg);
-
-      if (!queryModel.isDetailQuery()) {
-        for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) {
-          row[queryModel.getQueryMeasures().get(i).getQueryOrder()] =
-              msrAgg[queryExecuterProperties.measureStartIndex + i].get();
-        }
-        int index = 0;
-        for (int i = 0; i < queryModel.getDimAggregationInfo().size(); i++) {
-          DimensionAggregatorInfo dimensionAggregatorInfo =
-              queryModel.getDimAggregationInfo().get(i);
-          for (int j = 0; j < dimensionAggregatorInfo.getOrderList().size(); j++) {
-            row[dimensionAggregatorInfo.getOrderList().get(j)] = msrAgg[index++].get();
-          }
-        }
-        for (int i = 0; i < queryModel.getExpressions().size(); i++) {
-          row[queryModel.getExpressions().get(i).getQueryOrder()] =
-              ((MeasureAggregator) convertedResult[dimensionCount
-                  + queryExecuterProperties.aggExpressionStartIndex + i][columnIndex]).get();
-        }
-      } else {
-        QueryMeasure msr = null;
-        for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) {
-          msr = queryModel.getQueryMeasures().get(i);
-          if (msrAgg[queryExecuterProperties.measureStartIndex + i].isFirstTime()) {
-            row[msr.getQueryOrder()] = null;
-          } else {
-            Object msrVal;
-            switch (msr.getMeasure().getDataType()) {
-              case LONG:
-                msrVal = msrAgg[queryExecuterProperties.measureStartIndex + i].getLongValue();
-                break;
-              case DECIMAL:
-                msrVal = msrAgg[queryExecuterProperties.measureStartIndex + i].getBigDecimalValue();
-                break;
-              default:
-                msrVal = msrAgg[queryExecuterProperties.measureStartIndex + i].getDoubleValue();
-            }
-            row[msr.getQueryOrder()] = DataTypeUtil
-                .getMeasureDataBasedOnDataType(msrVal,msr.getMeasure().getDataType());
-          }
-        }
-      }
-      values.add(new CarbonValue(new MeasureAggregator[0]));
-      keys.add(new CarbonKey(row));
-    }
-    LOGGER.info("###########################################------ Total Number of records"
-            + resultDataA[0].length);
-    BatchResult chunkResult = new BatchResult();
-    chunkResult.setKeys(keys);
-    chunkResult.setValues(values);
-    return chunkResult;
-  }
-
-  private void fillDimensionAggValue(QueryModel queryModel, Object[][] surrogateResult,
-      int dimensionCount, int columnIndex, MeasureAggregator[] v) {
-    Iterator<DimensionAggregatorInfo> dimAggInfoIterator =
-        queryModel.getDimAggregationInfo().iterator();
-    DimensionAggregatorInfo dimensionAggregatorInfo = null;
-    List<String> partitionColumns = queryModel.getParitionColumns();
-    int rowIndex = -1;
-    int index = 0;
-    while (dimAggInfoIterator.hasNext()) {
-      dimensionAggregatorInfo = dimAggInfoIterator.next();
-      for (int j = 0; j < dimensionAggregatorInfo.getAggList().size(); j++) {
-        ++rowIndex;
-        if (!dimensionAggregatorInfo.getAggList().get(j)
-            .equals(CarbonCommonConstants.DISTINCT_COUNT)) {
-          v[index++] =
-              ((MeasureAggregator) surrogateResult[dimensionCount + rowIndex][columnIndex]);
-        } else if (partitionColumns.size() == 1 && partitionColumns
-            .contains(dimensionAggregatorInfo.getColumnName()) && dimensionAggregatorInfo
-            .getAggList().get(j).equals(CarbonCommonConstants.DISTINCT_COUNT)) {
-          double value =
-              ((MeasureAggregator) surrogateResult[dimensionCount + rowIndex][columnIndex])
-                  .getDoubleValue();
-
-          MeasureAggregator countAggregator = new CountAggregator();
-          countAggregator.setNewValue(value);
-          v[index++] = countAggregator;
-        } else {
-          if (surrogateResult[dimensionCount
-              + rowIndex][columnIndex] instanceof DistinctCountAggregator) {
-
-            Iterator<Integer> iterator =
-                ((DistinctCountAggregator) surrogateResult[dimensionCount + rowIndex][columnIndex])
-                    .getBitMap().iterator();
-
-            MeasureAggregator distinctCountAggregatorObjct = new DistinctStringCountAggregator();
-            while (iterator.hasNext()) {
-              String member = queryExecuterProperties.columnToDictionayMapping
-                  .get(dimensionAggregatorInfo.getDim().getColumnId())
-                  .getDictionaryValueForKey(iterator.next());
-              if (!member.equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
-                distinctCountAggregatorObjct.agg(member);
-              }
-            }
-            v[index++] = distinctCountAggregatorObjct;
-          } else {
-            v[index++] =
-                ((MeasureAggregator) surrogateResult[dimensionCount + rowIndex][columnIndex]);
-          }
-        }
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
index 0eb60ff..0ae6651 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
@@ -1,22 +1,24 @@
 package org.carbondata.query.carbon.result.preparator.impl;
 
+import java.util.List;
+
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.query.aggregator.MeasureAggregator;
 import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QueryMeasure;
 import org.carbondata.query.carbon.model.QueryModel;
 import org.carbondata.query.carbon.model.QuerySchemaInfo;
 import org.carbondata.query.carbon.result.BatchRawResult;
+import org.carbondata.query.carbon.result.BatchResult;
+import org.carbondata.query.carbon.result.ListBasedResultWrapper;
 import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 
 /**
  * It does not decode the dictionary.
  */
-public class RawQueryResultPreparatorImpl extends AbstractQueryResultPreparator<BatchRawResult> {
+public class RawQueryResultPreparatorImpl
+    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, Object> {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(RawQueryResultPreparatorImpl.class.getName());
@@ -33,7 +35,7 @@ public class RawQueryResultPreparatorImpl extends AbstractQueryResultPreparator<
         .toArray(new QueryDimension[queryModel.getQueryDimension().size()]));
     querySchemaInfo.setQueryMeasures(queryModel.getQueryMeasures()
         .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
-    int msrSize = queryExecuterProperties.measureAggregators.length;
+    int msrSize = queryExecuterProperties.measureDataTypes.length;
     int dimensionCount = queryModel.getQueryDimension().size();
     int[] queryOrder = new int[dimensionCount + msrSize];
     int[] queryReverseOrder = new int[dimensionCount + msrSize];
@@ -49,75 +51,34 @@ public class RawQueryResultPreparatorImpl extends AbstractQueryResultPreparator<
     querySchemaInfo.setQueryReverseOrder(queryReverseOrder);
   }
 
-  @Override public BatchRawResult prepareQueryResult(Result scannedResult) {
+  @Override public BatchResult prepareQueryResult(
+      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
     if ((null == scannedResult || scannedResult.size() < 1)) {
-      BatchRawResult batchRawResult = new BatchRawResult(new Object[0][0]);
+      BatchRawResult batchRawResult = new BatchRawResult();
       batchRawResult.setQuerySchemaInfo(querySchemaInfo);
       return batchRawResult;
     }
-    int msrSize = queryExecuterProperties.measureAggregators.length;
-    int totalNumberOfColumn = msrSize + 1;
-    Object[][] resultData = new Object[scannedResult.size()][totalNumberOfColumn];
-    int currentRow = 0;
-    ByteArrayWrapper key = null;
-    MeasureAggregator[] value = null;
+    int msrSize = queryExecuterProperties.measureDataTypes.length;
+    Object[][] resultData = new Object[scannedResult.size()][];
+    Object[] value;
+    Object[] row;
+    int counter = 0;
     while (scannedResult.hasNext()) {
-      key = scannedResult.getKey();
       value = scannedResult.getValue();
-      resultData[currentRow][0] = key;
-      for (int i = 0; i < msrSize; i++) {
-        resultData[currentRow][1 + i] = value[i];
+      row = new Object[msrSize + 1];
+      row[0] = scannedResult.getKey();
+      if(value != null) {
+        System.arraycopy(value, 0, row, 1, msrSize);
       }
-      currentRow++;
-    }
-
-    if (resultData.length > 0) {
-      resultData = encodeToRows(resultData);
+      resultData[counter] = row;
+      counter ++;
     }
-    BatchRawResult result = getResult(queryModel, resultData);
+    LOGGER.info("###########################---- Total Number of records" + scannedResult.size());
+    BatchRawResult result = new BatchRawResult();
+    result.setRows(resultData);
     result.setQuerySchemaInfo(querySchemaInfo);
     return result;
   }
 
-
-  private BatchRawResult getResult(QueryModel queryModel, Object[][] convertedResult) {
-
-    int msrCount = queryExecuterProperties.measureAggregators.length;
-    Object[][] resultDataA = new Object[1 + msrCount][convertedResult[0].length];
-
-    for (int columnIndex = 0; columnIndex < resultDataA[0].length; columnIndex++) {
-      resultDataA[0][columnIndex] = convertedResult[0][columnIndex];
-      MeasureAggregator[] msrAgg =
-          new MeasureAggregator[queryExecuterProperties.measureAggregators.length];
-
-      fillMeasureValueForAggGroupByQuery(queryModel, convertedResult, 1, columnIndex, msrAgg);
-
-      QueryMeasure msr = null;
-      for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) {
-        msr = queryModel.getQueryMeasures().get(i);
-        if (msrAgg[queryExecuterProperties.measureStartIndex + i].isFirstTime()) {
-          resultDataA[i + 1][columnIndex] = null;
-        } else {
-          Object msrVal;
-          switch (msr.getMeasure().getDataType()) {
-            case LONG:
-              msrVal = msrAgg[queryExecuterProperties.measureStartIndex + i].getLongValue();
-              break;
-            case DECIMAL:
-              msrVal = msrAgg[queryExecuterProperties.measureStartIndex + i].getBigDecimalValue();
-              break;
-            default:
-              msrVal = msrAgg[queryExecuterProperties.measureStartIndex + i].getDoubleValue();
-          }
-          resultDataA[i + 1][columnIndex] = DataTypeUtil
-              .getMeasureDataBasedOnDataType(msrVal,
-                  msr.getMeasure().getDataType());
-        }
-      }
-    }
-    LOGGER.info("###########################################------ Total Number of records"
-            + resultDataA[0].length);
-    return new BatchRawResult(resultDataA);
-  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java
index ca7c77a..0377580 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelFilterExecuterImpl.java
@@ -35,8 +35,6 @@ import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.util.MeasureAggregatorFactory;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.processor.BlocksChunkHolder;
 import org.carbondata.query.carbon.util.DataTypeUtil;
@@ -209,44 +207,25 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
       if (!msrColumnEvalutorInfo.isMeasureExistsInCurrentSlice()) {
         record[msrColumnEvalutorInfo.getRowIndex()] = msrColumnEvalutorInfo.getDefaultValue();
       } else {
-        if (msrColumnEvalutorInfo.isCustomMeasureValue()) {
-          MeasureAggregator aggregator = MeasureAggregatorFactory
-              .getAggregator(msrColumnEvalutorInfo.getAggregator(),
-                  msrColumnEvalutorInfo.getType());
-          aggregator.merge(
-              blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
-                  .getMeasureDataHolder().getReadableByteArrayValueByIndex(index));
-          switch (msrType) {
-            case LONG:
-              record[msrColumnEvalutorInfo.getRowIndex()] = aggregator.getLongValue();
-              break;
-            case DECIMAL:
-              record[msrColumnEvalutorInfo.getRowIndex()] = aggregator.getBigDecimalValue();
-              break;
-            default:
-              record[msrColumnEvalutorInfo.getRowIndex()] = aggregator.getDoubleValue();
-          }
-        } else {
-          Object msrValue;
-          switch (msrType) {
-            case LONG:
-              msrValue =
-                  blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
-                      .getMeasureDataHolder().getReadableLongValueByIndex(index);
-              break;
-            case DECIMAL:
-              msrValue =
-                  blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
-                      .getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-              break;
-            default:
-              msrValue =
-                  blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
-                      .getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-          }
-          record[msrColumnEvalutorInfo.getRowIndex()] = msrValue;
-
+        Object msrValue;
+        switch (msrType) {
+          case LONG:
+            msrValue =
+                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
+                    .getMeasureDataHolder().getReadableLongValueByIndex(index);
+            break;
+          case DECIMAL:
+            msrValue =
+                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
+                    .getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+            break;
+          default:
+            msrValue =
+                blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
+                    .getMeasureDataHolder().getReadableDoubleValueByIndex(index);
         }
+        record[msrColumnEvalutorInfo.getRowIndex()] = msrValue;
+
       }
     }
     row.setValues(record);
@@ -275,7 +254,7 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * Read the actual filter member by passing the dictionary value from
    * the forward dictionary cache which which holds column wise cache
    *
-   * @param dimColumnEvaluaatorInfo
+   * @param dimColumnEvaluatorInfo
    * @param dictionaryValue
    * @param forwardDictionary
    * @return

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
index 9cefbdb..e6877d5 100644
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
+++ b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
@@ -31,8 +31,6 @@ public class MeasureColumnResolvedFilterInfo implements Serializable {
 
   private int rowIndex = -1;
 
-  private boolean isCustomMeasureValue;
-
   private Object uniqueValue;
 
   private String aggregator;
@@ -59,14 +57,6 @@ public class MeasureColumnResolvedFilterInfo implements Serializable {
     this.rowIndex = rowIndex;
   }
 
-  public boolean isCustomMeasureValue() {
-    return isCustomMeasureValue;
-  }
-
-  public void setCustomMeasureValue(boolean isCustomMeasureValue) {
-    this.isCustomMeasureValue = isCustomMeasureValue;
-  }
-
   public Object getUniqueValue() {
     return uniqueValue;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/scanner/impl/CarbonKey.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/scanner/impl/CarbonKey.java b/core/src/main/java/org/carbondata/query/scanner/impl/CarbonKey.java
deleted file mode 100644
index 7f6d7f2..0000000
--- a/core/src/main/java/org/carbondata/query/scanner/impl/CarbonKey.java
+++ /dev/null
@@ -1,105 +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.carbondata.query.scanner.impl;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-/**
- * @author R00900208
- */
-public class CarbonKey implements Serializable, Comparable<CarbonKey> {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = -8773813519739848506L;
-
-  private Object[] key;
-
-  public CarbonKey(Object[] key) {
-    this.key = key;
-  }
-
-  /**
-   * @return the key
-   */
-  public Object[] getKey() {
-    return key;
-  }
-
-  public CarbonKey getSubKey(int size) {
-    Object[] crop = new Object[size];
-    System.arraycopy(key, 0, crop, 0, size);
-    return new CarbonKey(crop);
-  }
-
-  /* (non-Javadoc)
-   * @see java.lang.Object#hashCode()
-   */
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + Arrays.hashCode(key);
-    return result;
-  }
-
-  /* (non-Javadoc)
-   * @see java.lang.Object#equals(java.lang.Object)
-   */
-  @Override public boolean equals(Object obj) {
-    if (this == obj) {
-      return true;
-    }
-    if (obj == null) {
-      return false;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    CarbonKey other = (CarbonKey) obj;
-    if (!Arrays.equals(key, other.key)) {
-      return false;
-    }
-    return true;
-  }
-
-  @Override public String toString() {
-    return Arrays.toString(key);
-  }
-
-  @Override public int compareTo(CarbonKey other) {
-    Object[] oKey = other.key;
-
-    int l = 0;
-    for (int i = 0; i < key.length; i++) {
-      l = ((Comparable) key[i]).compareTo(oKey[i]);
-      if (l != 0) {
-        return l;
-      }
-    }
-
-    return 0;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/scanner/impl/CarbonValue.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/scanner/impl/CarbonValue.java b/core/src/main/java/org/carbondata/query/scanner/impl/CarbonValue.java
deleted file mode 100644
index 68e7226..0000000
--- a/core/src/main/java/org/carbondata/query/scanner/impl/CarbonValue.java
+++ /dev/null
@@ -1,87 +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.carbondata.query.scanner.impl;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class CarbonValue implements Serializable, Comparable<CarbonValue> {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 8034398963696130423L;
-
-  private MeasureAggregator[] values;
-
-  private int topNIndex;
-
-  public CarbonValue(MeasureAggregator[] values) {
-    this.values = values;
-  }
-
-  /**
-   * @return the values
-   */
-  public MeasureAggregator[] getValues() {
-    return values;
-  }
-
-  public CarbonValue merge(CarbonValue another) {
-    for (int i = 0; i < values.length; i++) {
-      values[i].merge(another.values[i]);
-    }
-    return this;
-  }
-
-  public void setTopNIndex(int index) {
-    this.topNIndex = index;
-  }
-
-  public void addGroup(CarbonKey key, CarbonValue value) {
-
-  }
-
-  public CarbonValue mergeKeyVal(CarbonValue another) {
-    return another;
-  }
-
-  @Override public String toString() {
-    return Arrays.toString(values);
-  }
-
-  @Override public int compareTo(CarbonValue o) {
-    return values[topNIndex].compareTo(o.values[topNIndex]);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof CarbonValue)) {
-      return false;
-    }
-    CarbonValue o = (CarbonValue)obj;
-    return values[topNIndex].equals(o.values[o.topNIndex]);
-  }
-
-  @Override public int hashCode() {
-    return values[topNIndex].hashCode();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
index 8ea0104..3d54d96 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
@@ -13,8 +13,8 @@ import org.carbondata.hadoop.readsupport.CarbonReadSupport;
 import org.carbondata.query.carbon.executor.QueryExecutorFactory;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchRawResult;
-import org.carbondata.query.carbon.result.iterator.ChunkRawRowIterartor;
+import org.carbondata.query.carbon.result.BatchResult;
+import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
 
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;
@@ -48,8 +48,8 @@ public class CarbonRecordReader<T> extends RecordReader<Void, T> {
     readSupport
         .intialize(queryModel.getProjectionColumns(), queryModel.getAbsoluteTableIdentifier());
     try {
-      carbonIterator = new ChunkRawRowIterartor(
-          (CarbonIterator<BatchRawResult>) QueryExecutorFactory.getQueryExecutor(queryModel)
+      carbonIterator = new ChunkRowIterator(
+          (CarbonIterator<BatchResult>) QueryExecutorFactory.getQueryExecutor(queryModel)
               .execute(queryModel));
     } catch (QueryExecutionException e) {
       throw new InterruptedException(e.getMessage());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index f728a32..b6f589d 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -17,17 +17,12 @@
 
 package org.apache.spark.sql
 
-import scala.collection.mutable.MutableList
-
-import org.apache.spark.sql.agg.{CarbonAverage, CarbonCount}
 import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, _}
 import org.apache.spark.sql.execution.command.tableModel
-import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
 import org.apache.spark.sql.types._
@@ -202,328 +197,3 @@ case class FakeCarbonCast(child: Literal, dataType: DataType)
 
   override def eval(input: InternalRow): Any = child.value
 }
-
-/**
- * A pattern that matches any number of project or filter operations on top of another relational
- * operator.  All filter operators are collected and their conditions are broken up and returned
- * together with the top project operator.
- * [[org.apache.spark.sql.catalyst.expressions.Alias Aliases]] are in-lined/substituted if
- * necessary.
- */
-object PhysicalOperation1 extends PredicateHelper {
-  type ReturnType = (Seq[NamedExpression], Seq[Expression], Option[Seq[Expression]],
-    Option[Seq[SortOrder]], Option[Expression], LogicalPlan)
-
-  def apply(plan: LogicalPlan): Option[ReturnType] = {
-    val (fields, filters, child, _, groupby, sortOrder, limit) =
-      collectProjectsAndFilters(plan)
-
-    Some((fields.getOrElse(child.output), filters, groupby, sortOrder, limit, child))
-  }
-
-  /**
-   * Collects projects and filters, in-lining/substituting aliases if necessary.  Here are two
-   * examples for alias in-lining/substitution.  Before:
-   * {{{
-   *   SELECT c1 FROM (SELECT key AS c1 FROM t1) t2 WHERE c1 > 10
-   *   SELECT c1 AS c2 FROM (SELECT key AS c1 FROM t1) t2 WHERE c1 > 10
-   * }}}
-   * After:
-   * {{{
-   *   SELECT key AS c1 FROM t1 WHERE key > 10
-   *   SELECT key AS c2 FROM t1 WHERE key > 10
-   * }}}
-   */
-  def collectProjectsAndFilters(plan: LogicalPlan):
-  (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan,
-    Map[Attribute, Expression], Option[Seq[Expression]],
-    Option[Seq[SortOrder]], Option[Expression]) = {
-    plan match {
-      case Project(fields, child) =>
-        val (_, filters, other, aliases, groupby, sortOrder, limit) = collectProjectsAndFilters(
-          child)
-        val substitutedFields = fields.map(substitute(aliases)).asInstanceOf[Seq[NamedExpression]]
-        (Some(substitutedFields), filters, other, collectAliases(
-          substitutedFields), groupby, sortOrder, limit)
-
-      case Filter(condition, child) =>
-        val (fields, filters, other, aliases, groupby, sortOrder, limit) =
-          collectProjectsAndFilters(child)
-        val substitutedCondition = substitute(aliases)(condition)
-        (fields, filters ++ splitConjunctivePredicates(
-          substitutedCondition), other, aliases, groupby, sortOrder, limit)
-
-      case Aggregate(groupingExpressions, aggregateExpressions, child) =>
-        val (fields, filters, other, aliases, _, sortOrder, limit) = collectProjectsAndFilters(
-          child)
-
-        var aggExps: Seq[AggregateExpression] = Nil
-        aggregateExpressions.foreach(v => {
-          val list = findAggreagateExpression(v)
-          aggExps = aggExps ++ list
-        })
-
-        (fields, filters, other, aliases ++ collectAliases(aggregateExpressions), Some(
-          aggregateExpressions), sortOrder, limit)
-      case Sort(order, _, child) =>
-        val (fields, filters, other, aliases, groupby, _, limit) = collectProjectsAndFilters(child)
-        val substitutedOrder = order.map(s => SortOrder(substitute(aliases)(s.child), s.direction))
-        (fields, filters, other, aliases, groupby, Some(substitutedOrder), limit)
-      case Limit(limitExpr, child) =>
-        val (fields, filters, other, aliases, groupby, sortOrder, _) = collectProjectsAndFilters(
-          child)
-        (fields, filters, other, aliases, groupby, sortOrder, Some(limitExpr))
-      case other =>
-        (None, Nil, other, Map.empty, None, None, None)
-    }
-  }
-
-  def findAggreagateExpression(expr: Expression): Seq[AggregateExpression] = {
-    val exprList = expr match {
-      case d: AggregateExpression => d :: Nil
-      case Alias(ref, name) => findAggreagateExpression(ref)
-      case other =>
-        var listout: Seq[AggregateExpression] = Nil
-
-        other.children.foreach(v => {
-          val list = findAggreagateExpression(v)
-          listout = listout ++ list
-        })
-        listout
-    }
-    exprList
-  }
-
-  def collectProjectsAndFilters1(plan: LogicalPlan):
-  (Option[Seq[NamedExpression]], Seq[Expression], LogicalPlan, Map[Attribute, Expression],
-    Option[Seq[Expression]], Option[Seq[SortOrder]], Option[Expression]) = {
-    plan match {
-      case Project(fields, child) =>
-        val (_, filters, other, aliases, groupby, sortOrder, limit) = collectProjectsAndFilters(
-          child)
-        val substitutedFields = fields.map(substitute(aliases)).asInstanceOf[Seq[NamedExpression]]
-        (Some(substitutedFields), filters, other, collectAliases(
-          substitutedFields), groupby, sortOrder, limit)
-
-      case Filter(condition, child) =>
-        val (fields, filters, other, aliases, groupby, sortOrder, limit) =
-          collectProjectsAndFilters(child)
-        val substitutedCondition = substitute(aliases)(condition)
-        (fields, filters ++ splitConjunctivePredicates(
-          substitutedCondition), other, aliases, groupby, sortOrder, limit)
-
-      case Aggregate(groupingExpressions, aggregateExpressions, child) =>
-        val (fields, filters, other, aliases, _, sortOrder, limit) = collectProjectsAndFilters(
-          child)
-        val aggExps = aggregateExpressions.map {
-          case Alias(ref, name) => ref
-          case others => others
-        }.filter {
-          case d: AggregateExpression => true
-          case _ => false
-        }
-        (fields, filters, other, aliases ++ collectAliases(aggregateExpressions), Some(
-          aggExps), sortOrder, limit)
-      case Sort(order, _, child) =>
-        val (fields, filters, other, aliases, groupby, _, limit) = collectProjectsAndFilters(child)
-        val substitutedOrder = order.map(s => SortOrder(substitute(aliases)(s.child), s.direction))
-        (fields, filters, other, aliases, groupby, Some(substitutedOrder), limit)
-      case Limit(limitExpr, child) =>
-        val (fields, filters, other, aliases, groupby, sortOrder, _) = collectProjectsAndFilters(
-          child)
-        (fields, filters, other, aliases, groupby, sortOrder, Some(limitExpr))
-      case other =>
-        (None, Nil, other, Map.empty, None, None, None)
-    }
-  }
-
-  private def collectAliases(fields: Seq[Expression]) = {
-    fields.collect {
-      case a@Alias(child, _) => a.toAttribute -> child
-    }.toMap
-  }
-
-  private def substitute(aliases: Map[Attribute, Expression])(expr: Expression) = {
-    expr.transform {
-      case a@Alias(ref: AttributeReference, name) =>
-        aliases.get(ref).map(Alias(_, name)(a.exprId, a.qualifiers)).getOrElse(a)
-
-      case a: AttributeReference =>
-        aliases.get(a).map(Alias(_, a.name)(a.exprId, a.qualifiers)).getOrElse(a)
-    }
-  }
-}
-
-case class PositionLiteral(expr: Expression, intermediateDataType: DataType)
-  extends LeafExpression with CodegenFallback {
-  override def dataType: DataType = expr.dataType
-
-  override def nullable: Boolean = false
-
-  type EvaluatedType = Any
-  var position = -1
-
-  def setPosition(pos: Int): Unit = position = pos
-
-  override def toString: String = s"PositionLiteral($position : $expr)"
-
-  override def eval(input: InternalRow): Any = {
-    if (position != -1) {
-      input.get(position, intermediateDataType)
-    } else {
-      expr.eval(input)
-    }
-  }
-}
-
-/**
- * Matches a logical aggregation that can be performed on distributed data in two steps.  The first
- * operates on the data in each partition performing partial aggregation for each group.  The second
- * occurs after the shuffle and completes the aggregation.
- *
- * This pattern will only match if all aggregate expressions can be computed partially and will
- * return the rewritten aggregation expressions for both phases.
- *
- * The returned values for this match are as follows:
- * - Grouping attributes for the final aggregation.
- * - Aggregates for the final aggregation.
- * - Grouping expressions for the partial aggregation.
- * - Partial aggregate expressions.
- * - Input to the aggregation.
- */
-object CarbonAggregation {
-  type ReturnType = (Seq[Expression], Seq[NamedExpression], LogicalPlan)
-
-  private def convertAggregatesForPushdown(convertUnknown: Boolean,
-      rewrittenAggregateExpressions: Seq[Expression],
-      oneAttr: AttributeReference) = {
-    if (canBeConvertedToCarbonAggregate(rewrittenAggregateExpressions)) {
-      var counter: Int = 0
-      var updatedExpressions = MutableList[Expression]()
-      rewrittenAggregateExpressions.foreach(v => {
-        val updated = convertAggregate(v, counter, convertUnknown, oneAttr)
-        updatedExpressions += updated
-        counter = counter + 1
-      })
-      updatedExpressions
-    } else {
-      rewrittenAggregateExpressions
-    }
-  }
-
-  def makePositionLiteral(expr: Expression, index: Int, dataType: DataType): PositionLiteral = {
-    val posLiteral = PositionLiteral(expr, dataType)
-    posLiteral.setPosition(index)
-    posLiteral
-  }
-
-  def convertAggregate(current: Expression,
-      index: Int,
-      convertUnknown: Boolean,
-      oneAttr: AttributeReference): Expression = {
-    if (!convertUnknown && canBeConverted(current)) {
-      current.transform {
-        case Average(attr: AttributeReference) =>
-          val convertedDataType = transformArrayType(attr)
-          CarbonAverage(makePositionLiteral(convertedDataType, index, convertedDataType.dataType))
-        case Average(Cast(attr: AttributeReference, dataType)) =>
-          val convertedDataType = transformArrayType(attr)
-          CarbonAverage(
-              makePositionLiteral(convertedDataType, index, convertedDataType.dataType))
-        case Count(Seq(s: Literal)) =>
-          CarbonCount(s, Some(makePositionLiteral(transformLongType(oneAttr), index, LongType)))
-        case Count(Seq(attr: AttributeReference)) =>
-          CarbonCount(makePositionLiteral(transformLongType(attr), index, LongType))
-        case Sum(attr: AttributeReference) =>
-          Sum(makePositionLiteral(attr, index, attr.dataType))
-        case Sum(Cast(attr: AttributeReference, dataType)) =>
-          Sum(Cast(makePositionLiteral(attr, index, attr.dataType), dataType))
-        case Min(attr: AttributeReference) => Min(makePositionLiteral(attr, index, attr.dataType))
-        case Min(Cast(attr: AttributeReference, dataType)) =>
-          Min(Cast(makePositionLiteral(attr, index, attr.dataType), dataType))
-        case Max(attr: AttributeReference) =>
-          Max(makePositionLiteral(attr, index, attr.dataType))
-        case Max(Cast(attr: AttributeReference, dataType)) =>
-          Max(Cast(makePositionLiteral(attr, index, attr.dataType), dataType))
-      }
-    } else {
-      current
-    }
-  }
-
-  def canBeConverted(current: Expression): Boolean = current match {
-    case Alias(AggregateExpression(Average(attr: AttributeReference), _, false), _) => true
-    case Alias(AggregateExpression(Average(Cast(attr: AttributeReference, _)), _, false), _) => true
-    case Alias(AggregateExpression(Count(Seq(s: Literal)), _, false), _) => true
-    case Alias(AggregateExpression(Count(Seq(attr: AttributeReference)), _, false), _) => true
-    case Alias(AggregateExpression(Sum(attr: AttributeReference), _, false), _) => true
-    case Alias(AggregateExpression(Sum(Cast(attr: AttributeReference, _)), _, false), _) => true
-    case Alias(AggregateExpression(Min(attr: AttributeReference), _, false), _) => true
-    case Alias(AggregateExpression(Min(Cast(attr: AttributeReference, _)), _, false), _) => true
-    case Alias(AggregateExpression(Max(attr: AttributeReference), _, false), _) => true
-    case Alias(AggregateExpression(Max(Cast(attr: AttributeReference, _)), _, false), _) => true
-    case _ => false
-  }
-
-  def transformArrayType(attr: AttributeReference): AttributeReference = {
-    AttributeReference(attr.name, ArrayType(DoubleType), attr.nullable, attr.metadata)(attr.exprId,
-      attr.qualifiers)
-  }
-
-  def transformLongType(attr: AttributeReference): AttributeReference = {
-    AttributeReference(attr.name, LongType, attr.nullable, attr.metadata)(attr.exprId,
-      attr.qualifiers)
-  }
-
-  /**
-   * There should be sync between carbonOperators validation and here. we should not convert to
-   * carbon aggregates if the validation does not satisfy.
-   */
-  def canBeConvertedToCarbonAggregate(expressions: Seq[Expression]): Boolean = {
-    val detailQuery = expressions.map {
-      case attr@AttributeReference(_, _, _, _) => true
-      case Alias(agg: AggregateExpression, name) => true
-      case _ => false
-    }.exists(!_)
-    !detailQuery
-  }
-
-  def unapply(plan: LogicalPlan): Option[ReturnType] = unapply((plan, false))
-
-  def unapply(combinedPlan: (LogicalPlan, Boolean)): Option[ReturnType] = {
-    val oneAttr = getOneAttribute(combinedPlan._1)
-    combinedPlan._1 match {
-      case Aggregate(groupingExpressions, aggregateExpressionsOrig, child) =>
-
-        // if detailed query dont convert aggregate expressions to Carbon Aggregate expressions
-        val aggregateExpressions =
-          if (combinedPlan._2) {
-            aggregateExpressionsOrig
-          }
-          else {
-            convertAggregatesForPushdown(false, aggregateExpressionsOrig, oneAttr)
-          }
-        Some((groupingExpressions, aggregateExpressions.asInstanceOf[Seq[NamedExpression]], child))
-      case _ => None
-    }
-  }
-
-  def getOneAttribute(plan: LogicalPlan): AttributeReference = {
-    var relation: LogicalRelation = null
-    plan collect {
-      case l: LogicalRelation => relation = l
-    }
-    if (relation != null) {
-      relation.output.find { p =>
-        p.dataType match {
-          case n: NumericType => true
-          case _ => false
-        }
-      }.getOrElse(relation.output.head)
-    } else {
-      null
-    }
-  }
-}
-
-



[47/56] [abbrv] incubator-carbondata git commit: [issue-581] fix the bug of load parse failure when tblname is in carbon's datatype in DDL (#704)

Posted by jb...@apache.org.
[issue-581] fix the bug of load parse failure when tblname is in carbon's datatype in DDL (#704)

* fix the bug of load parse failure when tblname is in carbon's datatype keywords in DDL

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

Branch: refs/heads/master
Commit: 5e2e66f1f79ec981c733b0c3ce7798230ab8ca81
Parents: 5c7b043
Author: Zhangshunyu <zh...@huawei.com>
Authored: Mon Jun 20 22:32:52 2016 +0800
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Mon Jun 20 20:02:52 2016 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonSqlParser.scala  | 33 +++----
 .../dataload/TestLoadTblNameIsKeyword.scala     | 96 ++++++++++++++++++++
 2 files changed, 110 insertions(+), 19 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e2e66f1/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index 56a8a8e..7474f4f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -23,6 +23,7 @@ import java.util.regex.{Matcher, Pattern}
 import scala.collection.JavaConverters._
 import scala.collection.mutable.LinkedHashSet
 import scala.language.implicitConversions
+import scala.util.matching.Regex
 
 import org.apache.hadoop.hive.ql.lib.Node
 import org.apache.hadoop.hive.ql.parse._
@@ -36,13 +37,10 @@ import org.apache.spark.sql.execution.datasources.DescribeCommand
 import org.apache.spark.sql.hive.HiveQlWrapper
 
 import org.carbondata.core.carbon.metadata.datatype.DataType
-import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.util.DataTypeUtil
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 import org.carbondata.spark.util.CommonUtil
 
-
-
 /**
  * Parser for All Carbon DDL, DML cases in Unified context
  */
@@ -73,7 +71,6 @@ class CarbonSqlParser()
   protected val DETAIL = Keyword("DETAIL")
   protected val DIMENSIONS = Keyword("DIMENSIONS")
   protected val DIMFOLDERPATH = Keyword("DIMFOLDERPATH")
-  protected val DOUBLE = Keyword("DOUBLE")
   protected val DROP = Keyword("DROP")
   protected val ESCAPECHAR = Keyword("ESCAPECHAR")
   protected val EXCLUDE = Keyword("EXCLUDE")
@@ -88,25 +85,17 @@ class CarbonSqlParser()
   protected val IN = Keyword("IN")
   protected val INCLUDE = Keyword("INCLUDE")
   protected val INPATH = Keyword("INPATH")
-  protected val INT = Keyword("INT")
-  protected val INTEGER = Keyword("INTEGER")
   protected val INTO = Keyword("INTO")
   protected val LEVELS = Keyword("LEVELS")
   protected val LIKE = Keyword("LIKE")
   protected val LOAD = Keyword("LOAD")
   protected val LOADS = Keyword("LOADS")
   protected val LOCAL = Keyword("LOCAL")
-  protected val LONG = Keyword("LONG")
   protected val MAPPED = Keyword("MAPPED")
   protected val MEASURES = Keyword("MEASURES")
   protected val MULTILINE = Keyword("MULTILINE")
   protected val COMPLEX_DELIMITER_LEVEL_1 = Keyword("COMPLEX_DELIMITER_LEVEL_1")
   protected val COMPLEX_DELIMITER_LEVEL_2 = Keyword("COMPLEX_DELIMITER_LEVEL_2")
-  protected val NUMERIC = Keyword("NUMERIC")
-  protected val ARRAY = Keyword("ARRAY")
-  protected val STRUCT = Keyword("STRUCT")
-  protected val BIGINT = Keyword("BIGINT")
-  protected val DECIMAL = Keyword("DECIMAL")
   protected val OPTIONS = Keyword("OPTIONS")
   protected val OUTPATH = Keyword("OUTPATH")
   protected val OVERWRITE = Keyword("OVERWRITE")
@@ -117,11 +106,9 @@ class CarbonSqlParser()
   protected val RELATION = Keyword("RELATION")
   protected val SCHEMAS = Keyword("SCHEMAS")
   protected val SHOW = Keyword("SHOW")
-  protected val STRING = Keyword("STRING")
   protected val TABLES = Keyword("TABLES")
   protected val TABLE = Keyword("TABLE")
   protected val TERMINATED = Keyword("TERMINATED")
-  protected val TIMESTAMP = Keyword("TIMESTAMP")
   protected val TYPE = Keyword("TYPE")
   protected val USE = Keyword("USE")
   protected val WHERE = Keyword("WHERE")
@@ -169,6 +156,11 @@ class CarbonSqlParser()
 
   }
 
+  import lexical.Identifier
+  implicit def regexToParser(regex: Regex): Parser[String] = acceptMatch(
+    s"identifier matching regex ${regex}",
+    { case Identifier(str) if regex.unapplySeq(str).isDefined => str }
+  )
   override def parse(input: String): LogicalPlan = synchronized {
     // Initialize the Keywords.
     initLexical
@@ -1131,7 +1123,9 @@ class CarbonSqlParser()
   protected lazy val dimCol: Parser[Field] = anyFieldDef
 
   protected lazy val primitiveTypes =
-    STRING | INTEGER | TIMESTAMP | NUMERIC | BIGINT | DECIMAL | INT | DOUBLE
+    "(?i)string".r ^^^ "string" | "(?i)integer".r ^^^ "integer" | "(?i)timestamp".r ^^^
+    "timestamp" | "(?i)numeric".r ^^^ "numeric" | "(?i)bigint".r ^^^ "bigint" |
+    "(?i)decimal".r ^^^ "decimal" | "(?i)int".r ^^^ "int" | "(?i)double".r ^^^ "double"
   protected lazy val nestedType: Parser[Field] = structFieldType | arrayFieldType |
     primitiveFieldType
 
@@ -1148,7 +1142,7 @@ class CarbonSqlParser()
     }
 
   protected lazy val arrayFieldType: Parser[Field] =
-    (ARRAY ~> "<" ~> nestedType <~ ">") ^^ {
+    (("(?i)array".r ^^^ "array") ~> "<" ~> nestedType <~ ">") ^^ {
       case e1 =>
         Field("unknown", Some("array"), Some("unknown"),
           Some(List(Field("val", e1.dataType, Some("val"),
@@ -1156,14 +1150,15 @@ class CarbonSqlParser()
     }
 
   protected lazy val structFieldType: Parser[Field] =
-    (STRUCT ~> "<" ~> repsep(anyFieldDef, ",") <~ ">") ^^ {
+    (("(?i)struct".r ^^^ "struct") ~> "<" ~> repsep(anyFieldDef, ",") <~ ">") ^^ {
       case e1 =>
         Field("unknown", Some("struct"), Some("unknown"), Some(e1))
     }
 
   protected lazy val measureCol: Parser[Field] =
-    (ident | stringLit) ~ (INTEGER | NUMERIC | BIGINT | DECIMAL).? ~ (AS ~> (ident | stringLit)).? ~
-      (IN ~> (ident | stringLit)).? ^^ {
+    (ident | stringLit) ~ ("(?i)integer".r ^^^ "integer" | "(?i)numeric".r ^^^ "numeric" |
+      "(?i)bigint".r ^^^ "bigint" | "(?i)decimal".r ^^^ "decimal").? ~
+      (AS ~> (ident | stringLit)).? ~ (IN ~> (ident | stringLit)).? ^^ {
       case e1 ~ e2 ~ e3 ~ e4 => Field(e1, e2, e3, Some(null))
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e2e66f1/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadTblNameIsKeyword.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadTblNameIsKeyword.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadTblNameIsKeyword.scala
new file mode 100644
index 0000000..6e52fb8
--- /dev/null
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadTblNameIsKeyword.scala
@@ -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.carbondata.integration.spark.testsuite.dataload
+
+import java.io.File
+
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+import org.apache.spark.sql.Row
+
+/**
+  * Test Class for data loading into table whose name is key word
+  *
+  */
+class TestLoadTblNameIsKeyword extends QueryTest with BeforeAndAfterAll {
+  def currentPath: String = new File(this.getClass.getResource("/").getPath + "/../../")
+    .getCanonicalPath
+  val testData = new File(currentPath + "/../../examples/src/main/resources/dimSample.csv")
+    .getCanonicalPath
+  override def beforeAll {
+    sql("drop table if exists STRING")
+    sql("drop table if exists DoUbLe")
+    sql("drop table if exists timestamp")
+    sql("""
+          CREATE TABLE IF NOT EXISTS STRING
+          (id Int, name String, city String)
+          STORED BY 'org.apache.carbondata.format'
+        """)
+    sql("""
+          CREATE TABLE IF NOT EXISTS DoUbLe
+          (id Int, name String, city String)
+          STORED BY 'org.apache.carbondata.format'
+        """)
+    sql("""
+          CREATE TABLE IF NOT EXISTS timestamp
+          (id Int, name String, city String)
+          STORED BY 'org.apache.carbondata.format'
+        """)
+  }
+
+  test("test load data whose name is a keyword of data type") {
+    sql(s"""
+          LOAD DATA LOCAL INPATH '$testData' into table STRING
+        """)
+    checkAnswer(
+      sql("""
+            SELECT count(*) from STRING
+          """),
+      Seq(Row(20)))
+  }
+
+  test("test case in-sensitiveness") {
+    sql(s"""
+          LOAD DATA LOCAL INPATH '$testData' into table DoUbLe
+        """)
+    checkAnswer(
+      sql("""
+            SELECT count(*) from DoUbLe
+          """),
+      Seq(Row(20)))
+  }
+
+  test("test other ddl whose table name a keyword of data type") {
+    sql("describe timestamp")
+    sql(s"""
+          LOAD DATA LOCAL INPATH '$testData' into table timestamp
+        """)
+    sql("show segments for table timestamp")
+    sql("delete segments from table timestamp where starttime before '2099-10-01 18:00:00'")
+    sql("clean files for table timestamp")
+  }
+
+  override def afterAll {
+    sql("drop table STRING")
+    sql("drop table DoUbLe")
+    sql("drop table timestamp")
+  }
+}


[11/56] [abbrv] incubator-carbondata git commit: Update SQL planning in carbon-spark (#682)

Posted by jb...@apache.org.
Update SQL planning in carbon-spark (#682)

* [Issue-660] Show segments query should not fail, if table name is case insensitive (#662)

* Show segments should not fail, if table name is case insensitive

* Corrected test case

* [issue-656] fix load data when int column contains integer.min_value (#657)

* load data when int column contains min Integer

* fixed test case

*  fix test bigint

*  fix test bigint

* removed no used DATA_BIGINT case

* removed no used condition for unCompressMaxMin

* [issue- 664] select count(joinDate) from table_x is failing for direct dictionary column (#665)

* Supported Spark 1.6 by changing aggregation interfaces

* Fixed compile issue after rebase

* optmizing the flow with unsafe row

* Fixed bugs in push up

* Fixed compiler issues after rebasing

* Fixed merging issue after rebase

* Fixed scan query pushdown

* keep pushup strategy only

* keep only on QueryRDD

* rename QueryRDD to ScanRDD and clean up code

* fix scalastyle


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

Branch: refs/heads/master
Commit: a83dba3433525eaf2f255912184a6e1a6d7dbdea
Parents: ead0076
Author: Jacky Li <ja...@huawei.com>
Authored: Fri Jun 17 16:02:55 2016 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Fri Jun 17 13:32:55 2016 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonContext.scala    |   5 +-
 .../org/apache/spark/sql/CarbonOperators.scala  | 676 ++++---------------
 .../apache/spark/sql/CarbonRawOperators.scala   | 332 ---------
 .../spark/sql/execution/joins/CarbonJoins.scala | 140 ----
 .../spark/sql/hive/CarbonRawStrategies.scala    | 217 ------
 .../spark/sql/hive/CarbonSQLDialect.scala       |  42 ++
 .../spark/sql/hive/CarbonStrategies.scala       | 505 +++++---------
 .../apache/spark/sql/hive/CarbonStrategy.scala  |  54 --
 .../carbondata/spark/rdd/CarbonQueryRDD.scala   | 241 -------
 .../spark/rdd/CarbonRawQueryRDD.scala           | 128 ----
 .../carbondata/spark/rdd/CarbonScanRDD.scala    | 219 ++++++
 .../testsuite/joinquery/EquiJoinTestCase.scala  |  41 --
 12 files changed, 566 insertions(+), 2034 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
index 2bf50da..ffc5655 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
@@ -55,7 +55,10 @@ class CarbonContext(val sc: SparkContext, val storePath: String) extends HiveCon
 
   protected[sql] override def getSQLDialect(): ParserDialect = new CarbonSQLDialect(this)
 
-  experimental.extraStrategies = CarbonStrategy.getStrategy(self)
+  experimental.extraStrategies = {
+    val carbonStrategy = new CarbonStrategies(self)
+    Seq(carbonStrategy.CarbonTableScan, carbonStrategy.DDLStrategies)
+  }
 
   @transient
   val LOGGER = LogServiceFactory.getLogService(CarbonContext.getClass.getName)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index 8796707..cb20246 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -23,278 +23,70 @@ import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
 
 import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.Job
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.agg.{CarbonAverage, CarbonCount}
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{GenericInternalRow, _}
-import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, Max, Min, Sum}
-import org.apache.spark.sql.catalyst.util.GenericArrayData
+import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
 import org.apache.spark.sql.execution.LeafNode
 import org.apache.spark.sql.hive.CarbonMetastoreCatalog
 import org.apache.spark.sql.types.{DataType, Decimal}
 import org.apache.spark.unsafe.types.UTF8String
 
-import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.carbon.AbsoluteTableIdentifier
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.util.CarbonProperties
-import org.carbondata.hadoop.CarbonInputFormat
-import org.carbondata.query.aggregator.MeasureAggregator
-import org.carbondata.query.aggregator.impl.avg.AbstractAvgAggregator
-import org.carbondata.query.aggregator.impl.count.CountAggregator
-import org.carbondata.query.aggregator.impl.max.{MaxAggregator, MaxBigDecimalAggregator, MaxLongAggregator}
-import org.carbondata.query.aggregator.impl.min.{MinAggregator, MinBigDecimalAggregator, MinLongAggregator}
-import org.carbondata.query.aggregator.impl.sum.{SumBigDecimalAggregator, SumDoubleAggregator, SumLongAggregator}
-import org.carbondata.query.carbon.model.{CarbonQueryPlan, QueryDimension, QueryMeasure, QueryModel, SortOrderType}
-import org.carbondata.query.carbon.result.RowResult
-import org.carbondata.query.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
-import org.carbondata.query.expression.arithmetic.{AddExpression, DivideExpression, MultiplyExpression, SubstractExpression}
-import org.carbondata.query.expression.conditional._
-import org.carbondata.query.expression.logical.{AndExpression, OrExpression}
-import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
-import org.carbondata.spark.{KeyVal, KeyValImpl}
-import org.carbondata.spark.rdd.CarbonQueryRDD
-import org.carbondata.spark.util.{CarbonScalaUtil, QueryPlanUtil}
-
-case class CarbonTableScan(
-    var attributes: Seq[Attribute],
-    relation: CarbonRelation,
-    dimensionPredicates: Seq[Expression],
-    aggExprs: Option[Seq[Expression]],
-    sortExprs: Option[Seq[SortOrder]],
-    limitExpr: Option[Expression],
-    isGroupByPresent: Boolean,
-    detailQuery: Boolean = false)(@transient val oc: SQLContext)
-  extends LeafNode {
-
-  val cubeName = relation.cubeName
-  val carbonTable = relation.metaData.carbonTable
+import org.carbondata.query.carbon.model._
+import org.carbondata.spark.{CarbonFilters, RawKey, RawKeyImpl}
+import org.carbondata.spark.rdd.CarbonScanRDD
+
+case class CarbonScan(
+    var attributesRaw: Seq[Attribute],
+    relationRaw: CarbonRelation,
+    dimensionPredicatesRaw: Seq[Expression],
+    aggExprsRaw: Option[Seq[Expression]],
+    useBinaryAggregator: Boolean)(@transient val ocRaw: SQLContext) extends LeafNode {
+  val carbonTable = relationRaw.metaData.carbonTable
   val selectedDims = scala.collection.mutable.MutableList[QueryDimension]()
   val selectedMsrs = scala.collection.mutable.MutableList[QueryMeasure]()
-  var outputColumns = scala.collection.mutable.MutableList[Attribute]()
-  var extraPreds: Seq[Expression] = Nil
-  val allDims = new scala.collection.mutable.HashSet[String]()
-  @transient val carbonCatalog = sqlContext.catalog.asInstanceOf[CarbonMetastoreCatalog]
-
-  def processAggregateExpr(plan: CarbonQueryPlan,
-      currentAggregate: AggregateExpression,
-      queryOrder: Int,
-      aggCount: Int): Int = {
-    currentAggregate match {
-      case AggregateExpression(Sum(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.SUM)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims.nonEmpty) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "sum", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case AggregateExpression(
-        Sum(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.SUM)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims.nonEmpty) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "sum", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case AggregateExpression(
-        CarbonCount(p@PositionLiteral(attr: AttributeReference, _), None), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.COUNT)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims.nonEmpty) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "count", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case AggregateExpression(
-        CarbonCount(lt: Literal, Some(p@PositionLiteral(attr: AttributeReference, _))), _, false)
-        if lt.value == "*" || lt.value == 1 =>
-        outputColumns += attr
-        val m1 = new QueryMeasure("count(*)")
-        m1.setAggregateFunction(CarbonCommonConstants.COUNT)
-        m1.setQueryOrder(queryOrder)
-        plan.addMeasure(m1)
-        plan.setCountStartQuery(true)
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
+  @transient val carbonCatalog = ocRaw.catalog.asInstanceOf[CarbonMetastoreCatalog]
 
-      case AggregateExpression(
-        CarbonAverage(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.AVERAGE)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims.nonEmpty) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "avg", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case AggregateExpression(
-        CarbonAverage(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.AVERAGE)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims.nonEmpty) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "avg", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case AggregateExpression(Min(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.MIN)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims != null) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "min", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
+  val attributesNeedToDecode = new java.util.HashSet[AttributeReference]()
+  val unprocessedExprs = new ArrayBuffer[Expression]()
 
-      case AggregateExpression(
-        Min(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.MIN)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims != null) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "min", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case AggregateExpression(Max(p@PositionLiteral(attr: AttributeReference, _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.MAX)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims.nonEmpty) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "max", d1.getQueryOrder)
-          }
-        }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case AggregateExpression(
-        Max(Cast(p@PositionLiteral(attr: AttributeReference, _), _)), _, false) =>
-        outputColumns += attr
-        val msrs = selectedMsrs.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (msrs.nonEmpty) {
-          val m1 = new QueryMeasure(attr.name)
-          m1.setAggregateFunction(CarbonCommonConstants.MAX)
-          m1.setQueryOrder(queryOrder)
-          plan.addMeasure(m1)
-        } else {
-          val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-          if (dims.nonEmpty) {
-            val d1 = new QueryDimension(attr.name)
-            d1.setQueryOrder(queryOrder)
-            plan.addAggDimAggInfo(d1.getColumnName, "max", d1.getQueryOrder)
-          }
+  val buildCarbonPlan: CarbonQueryPlan = {
+    val plan: CarbonQueryPlan = new CarbonQueryPlan(relationRaw.schemaName, relationRaw.tableName)
+
+    val dimensions = carbonTable.getDimensionByTableName(carbonTable.getFactTableName)
+    val measures = carbonTable.getMeasureByTableName(carbonTable.getFactTableName)
+    val dimAttr = new Array[Attribute](dimensions.size())
+    val msrAttr = new Array[Attribute](measures.size())
+    attributesRaw.foreach { attr =>
+      val carbonDimension =
+        carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
+      if(carbonDimension != null) {
+        dimAttr(dimensions.indexOf(carbonDimension)) = attr
+      } else {
+        val carbonMeasure =
+          carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name)
+        if(carbonMeasure != null) {
+          msrAttr(measures.indexOf(carbonMeasure)) = attr
         }
-        p.setPosition(queryOrder + aggCount)
-        queryOrder + 1
-
-      case _ => throw new
-          Exception("Some Aggregate functions cannot be pushed, force to detailequery")
+      }
     }
-  }
-
-  val buildCarbonPlan: CarbonQueryPlan = {
-    val plan: CarbonQueryPlan = new CarbonQueryPlan(relation.schemaName, relation.cubeName)
 
+    attributesRaw = dimAttr.filter(f => f != null) ++ msrAttr.filter(f => f != null)
 
-    var forceDetailedQuery = detailQuery
     var queryOrder: Integer = 0
-    attributes.map(
-      attr => {
-        val carbonDimension = carbonTable.getDimensionByName(carbonTable.getFactTableName
-          , attr.name)
+    attributesRaw.foreach { attr =>
+        val carbonDimension =
+          carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
         if (carbonDimension != null) {
-          allDims += attr.name
           val dim = new QueryDimension(attr.name)
           dim.setQueryOrder(queryOrder)
           queryOrder = queryOrder + 1
           selectedDims += dim
         } else {
-          val carbonMeasure = carbonTable.getMeasureByName(carbonTable.getFactTableName
-            , attr.name)
+          val carbonMeasure =
+            carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name)
           if (carbonMeasure != null) {
             val m1 = new QueryMeasure(attr.name)
             m1.setQueryOrder(queryOrder)
@@ -302,271 +94,108 @@ case class CarbonTableScan(
             selectedMsrs += m1
           }
         }
-      })
-    queryOrder = 0
-
-    // It is required to calculate as spark aggregators uses joined row with the current aggregates.
-    val aggCount = aggExprs match {
-      case Some(a: Seq[Expression]) =>
-        a.map {
-          case Alias(AggregateExpression(CarbonAverage(_), _, _), name) => 2
-          case Alias(agg: AggregateExpression, name) => 1
-          case _ => 0
-        }.reduceLeftOption((left, right) => left + right).getOrElse(0)
-      case _ => 0
-    }
-    // Separately handle group by columns, known or unknown partial aggregations and other
-    // expressions. All single column & known aggregate expressions will use native aggregates for
-    // measure and dimensions
-    // Unknown aggregates & Expressions will use custom aggregator
-
-    aggExprs match {
-      case Some(a: Seq[Expression]) if !forceDetailedQuery =>
-        a.foreach {
-          case attr@AttributeReference(_, _, _, _) => // Add all the references to carbon query
-            addCarbonColumn(attr)
-            outputColumns += attr
-          case al@ Alias(agg: AggregateExpression, name) =>
-            queryOrder = processAggregateExpr(plan, agg, queryOrder, aggCount)
-          case _ => forceDetailedQuery = true
-        }
-      case _ => forceDetailedQuery = true
-    }
-
-    def addCarbonColumn(attr: Attribute): Unit = {
-      val carbonDimension = selectedDims
-        .filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-      if (carbonDimension.nonEmpty) {
-        val dim = new QueryDimension(attr.name)
-        dim.setQueryOrder(queryOrder)
-        plan.addDimension(dim)
-        queryOrder = queryOrder + 1
-      } else {
-        val carbonMeasure = selectedMsrs
-          .filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-        if (carbonMeasure.nonEmpty) {
-          // added by vishal as we are adding for dimension so need to add to measure list
-          // Carbon does not support group by on measure column so throwing exception to
-          // make it detail query
-          throw new
-              Exception("Some Aggregate functions cannot be pushed, force to detailequery")
-        }
-        else {
-          // Some unknown attribute name is found. this may be a derived column.
-          // So, let's fall back to detailed query flow
-          throw new Exception(
-            "Some attributes referred looks derived columns. So, force to detailequery " +
-            attr.name)
-        }
-      }
-    }
-
-    if (forceDetailedQuery) {
-      // First clear the model if Msrs, Expressions and AggDimAggInfo filled
-      plan.getDimensions.clear()
-      plan.getMeasures.clear()
-      plan.getDimAggregatorInfos.clear()
-
-      // Fill the selected dimensions & measures obtained from
-      // attributes to query plan  for detailed query
-      selectedDims.foreach(plan.addDimension)
-      selectedMsrs.foreach(plan.addMeasure)
-    }
-    else {
-      attributes.foreach { attr =>
-        if (!outputColumns.exists(_.name.equals(attr.name))) {
-          addCarbonColumn(attr)
-          outputColumns += attr
-        }
       }
-      attributes = outputColumns
-    }
-
-    val orderList = new ArrayList[QueryDimension]()
-
-    var allSortExprPushed = true
-    sortExprs match {
-      case Some(a: Seq[SortOrder]) =>
-        a.foreach {
-          case SortOrder(Sum(attr: AttributeReference), order) => plan.getMeasures
-            .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name)).head
-            .setSortOrder(getSortDirection(order))
-          case SortOrder(CarbonCount(attr: AttributeReference, _), order) => plan.getMeasures
-            .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name)).head
-            .setSortOrder(getSortDirection(order))
-          case SortOrder(CarbonAverage(attr: AttributeReference), order) => plan.getMeasures
-            .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name)).head
-            .setSortOrder(getSortDirection(order))
-          case SortOrder(attr: AttributeReference, order) =>
-            val dim = plan.getDimensions
-              .asScala.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-            if (dim.nonEmpty) {
-              dim.head.setSortOrder(getSortDirection(order))
-              orderList.add(dim.head)
-            } else {
-              allSortExprPushed = false
+    // Just find out that any aggregation functions are present on dimensions.
+    aggExprsRaw match {
+      case Some(aggExprs) =>
+        aggExprs.foreach {
+          case Alias(agg: AggregateExpression, name) =>
+            agg.collect {
+              case attr: AttributeReference =>
+                val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
+                if(dims.nonEmpty) {
+                  plan.addAggDimAggInfo(dims.head.getColumnName,
+                    dims.head.getAggregateFunction,
+                    dims.head.getQueryOrder)
+                }
             }
-          case _ => allSortExprPushed = false;
+          case _ =>
         }
       case _ =>
     }
 
-    plan.setSortedDimemsions(orderList)
+    // Fill the selected dimensions & measures obtained from
+    // attributes to query plan  for detailed query
+    selectedDims.foreach(plan.addDimension)
+    selectedMsrs.foreach(plan.addMeasure)
 
-    // limit can be pushed down only if sort is not present or all sort expressions are pushed
-    if (sortExprs.isEmpty && forceDetailedQuery) {
-      limitExpr match {
-        case Some(IntegerLiteral(limit)) =>
-          // if (plan.getMeasures.size() == 0 && plan.getDimAggregatorInfos.size() == 0) {
-          plan.setLimit(limit)
-        // }
-        case _ =>
-      }
-    }
-    plan.setDetailQuery(forceDetailedQuery)
+    plan.setSortedDimemsions(new ArrayList[QueryDimension])
+
+    plan.setRawDetailQuery(true)
     plan.setOutLocationPath(
       CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS))
     plan.setQueryId(System.nanoTime() + "")
-    if (dimensionPredicates.nonEmpty) {
-      val exps = preProcessExpressions(dimensionPredicates)
-      val expressionVal = transformExpression(exps.head)
-      // adding dimension used in expression in querystats
-      expressionVal.getChildren.asScala.filter { x => x.isInstanceOf[CarbonColumnExpression] }
-        .map { y => allDims += y.asInstanceOf[CarbonColumnExpression].getColumnName }
-      plan.setFilterExpression(expressionVal)
-    }
+    processFilterExpressions(plan)
     plan
   }
 
-  def preProcessExpressions(expressions: Seq[Expression]): Seq[Expression] = {
-    expressions match {
-      case left :: right :: rest => preProcessExpressions(List(And(left, right)) ::: rest)
-      case List(left, right) => List(And(left, right))
-
-      case _ => expressions
-    }
-  }
-
-  def transformExpression(expr: Expression): CarbonExpression = {
-    expr match {
-      case Or(left, right) => new
-          OrExpression(transformExpression(left), transformExpression(right))
-      case And(left, right) => new
-          AndExpression(transformExpression(left), transformExpression(right))
-      case EqualTo(left, right) => new
-          EqualToExpression(transformExpression(left), transformExpression(right))
-      case Not(EqualTo(left, right)) => new
-          NotEqualsExpression(transformExpression(left), transformExpression(right))
-      case IsNotNull(child) => new
-          NotEqualsExpression(transformExpression(child), transformExpression(Literal(null)))
-      case Not(In(left, right)) => new NotInExpression(transformExpression(left),
-        new ListExpression(right.map(transformExpression).asJava))
-      case In(left, right) => new InExpression(transformExpression(left),
-        new ListExpression(right.map(transformExpression).asJava))
-      case Add(left, right) => new
-          AddExpression(transformExpression(left), transformExpression(right))
-      case Subtract(left, right) => new
-          SubstractExpression(transformExpression(left), transformExpression(right))
-      case Multiply(left, right) => new
-          MultiplyExpression(transformExpression(left), transformExpression(right))
-      case Divide(left, right) => new
-          DivideExpression(transformExpression(left), transformExpression(right))
-      case GreaterThan(left, right) => new
-          GreaterThanExpression(transformExpression(left), transformExpression(right))
-      case LessThan(left, right) => new
-          LessThanExpression(transformExpression(left), transformExpression(right))
-      case GreaterThanOrEqual(left, right) => new
-          GreaterThanEqualToExpression(transformExpression(left), transformExpression(right))
-      case LessThanOrEqual(left, right) => new
-          LessThanEqualToExpression(transformExpression(left), transformExpression(right))
-      // convert StartWith('abc') or like(col 'abc%') to col >= 'abc' and col < 'abd'
-      case StartsWith(left, right @ Literal(pattern, dataType)) if (pattern.toString.size > 0) =>
-        val l = new GreaterThanEqualToExpression(
-          transformExpression(left), transformExpression(right))
-        val value = pattern.toString
-        val maxValueLimit = value.substring(0, value.length - 1) +
-          (value.charAt(value.length - 1).toInt + 1).toChar
-        val r = new LessThanExpression(
-          transformExpression(left),
-            new CarbonLiteralExpression(maxValueLimit,
-              CarbonScalaUtil.convertSparkToCarbonDataType(dataType)))
-        new AndExpression(l, r)
-      case AttributeReference(name, dataType, _, _) => new CarbonColumnExpression(name.toString,
-        CarbonScalaUtil.convertSparkToCarbonDataType(dataType))
-      case Literal(name, dataType) => new
-          CarbonLiteralExpression(name, CarbonScalaUtil.convertSparkToCarbonDataType(dataType))
-      case Cast(left, right) if !left.isInstanceOf[Literal] => transformExpression(left)
-      case aggExpr: AggregateExpression =>
-          throw new UnsupportedOperationException(s"Cannot evaluate expression: $aggExpr")
-      case _ =>
-        new SparkUnknownExpression(expr.transform {
-          case AttributeReference(name, dataType, _, _) =>
-            CarbonBoundReference(new CarbonColumnExpression(name.toString,
-              CarbonScalaUtil.convertSparkToCarbonDataType(dataType)), dataType, expr.nullable)
-        })
+  def processFilterExpressions(plan: CarbonQueryPlan) {
+    if (dimensionPredicatesRaw.nonEmpty) {
+      val expressionVal = CarbonFilters.processExpression(
+        dimensionPredicatesRaw,
+        attributesNeedToDecode,
+        unprocessedExprs,
+        carbonTable)
+      expressionVal match {
+        case Some(ce) =>
+          // adding dimension used in expression in querystats
+          plan.setFilterExpression(ce)
+        case _ =>
+      }
     }
+    processExtraAttributes(plan)
   }
 
-  private def getSortDirection(sort: SortDirection) = {
-    sort match {
-      case Ascending => SortOrderType.ASC
-      case Descending => SortOrderType.DSC
+  private def processExtraAttributes(plan: CarbonQueryPlan) {
+    if (attributesNeedToDecode.size() > 0) {
+      val attributeOut = new ArrayBuffer[Attribute]() ++ attributesRaw
+
+      attributesNeedToDecode.asScala.map { attr =>
+        val dims = plan.getDimensions.asScala.filter(f => f.getColumnName.equals(attr.name))
+        val msrs = plan.getMeasures.asScala.filter(f => f.getColumnName.equals(attr.name))
+        var order = plan.getDimensions.size() + plan.getMeasures.size()
+        if (dims.isEmpty && msrs.isEmpty) {
+          val dimension = carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
+          if (dimension != null) {
+            val qDim = new QueryDimension(dimension.getColName)
+            qDim.setQueryOrder(order)
+            plan.addDimension(qDim)
+            attributeOut += attr
+            order += 1
+          } else {
+            val measure = carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name)
+            if (measure != null) {
+              val qMsr = new QueryMeasure(measure.getColName)
+              qMsr.setQueryOrder(order)
+              plan.addMeasure(qMsr)
+              order += 1
+              attributeOut += attr
+            }
+          }
+        }
+      }
+      attributesRaw = attributeOut
     }
   }
 
 
-  def addPushdownFilters(keys: Seq[Expression], filters: Array[Array[Expression]],
-      conditions: Option[Expression]) {
-
-    // TODO Values in the IN filter is duplicate. replace the list with set
-    val buffer = new ArrayBuffer[Expression]
-    keys.zipWithIndex.foreach { a =>
-      buffer += In(a._1, filters(a._2)).asInstanceOf[Expression]
-    }
-
-    // Let's not pushdown condition. Only filter push down is sufficient.
-    // Conditions can be applied on hash join result.
-    val cond = if (buffer.size > 1) {
-      val e = buffer.remove(0)
-      buffer.fold(e)(And(_, _))
-    } else {
-      buffer.asJava.get(0)
-    }
-
-    extraPreds = Seq(cond)
-  }
-
-  def inputRdd: CarbonQueryRDD[CarbonKey, CarbonValue] = {
-    val LOG = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-    // Update the FilterExpressions with extra conditions added through join pushdown
-    if (extraPreds.nonEmpty) {attributes
-      val exps = preProcessExpressions(extraPreds)
-      val expressionVal = transformExpression(exps.head)
-      val oldExpressionVal = buildCarbonPlan.getFilterExpression
-      if (null == oldExpressionVal) {
-        buildCarbonPlan.setFilterExpression(expressionVal)
-      } else {
-        buildCarbonPlan.setFilterExpression(new AndExpression(oldExpressionVal, expressionVal))
-      }
-    }
+  def inputRdd: CarbonScanRDD[Array[Any], Any] = {
 
     val conf = new Configuration()
     val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-
+    buildCarbonPlan.getDimAggregatorInfos.clear()
     val model = QueryModel.createModel(
       absoluteTableIdentifier, buildCarbonPlan, carbonTable)
-    val kv: KeyVal[CarbonKey, CarbonValue] = new KeyValImpl()
+    val kv: RawKey[Array[Any], Any] = new RawKeyImpl()
     // setting queryid
-    buildCarbonPlan.setQueryId(oc.getConf("queryId", System.nanoTime() + ""))
-
-    LOG.info("Selected Table to Query ****** "
-             + model.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
-
-    val cubeCreationTime = carbonCatalog.getCubeCreationTime(relation.schemaName, cubeName)
-    val schemaLastUpdatedTime =
-      carbonCatalog.getSchemaLastUpdatedTime(relation.schemaName, cubeName)
-    val big = new CarbonQueryRDD(
-      oc.sparkContext,
+    buildCarbonPlan.setQueryId(ocRaw.getConf("queryId", System.nanoTime() + ""))
+
+    val cubeCreationTime = carbonCatalog
+      .getCubeCreationTime(relationRaw.schemaName, relationRaw.tableName)
+    val schemaLastUpdatedTime = carbonCatalog
+      .getSchemaLastUpdatedTime(relationRaw.schemaName, relationRaw.tableName)
+    val big = new CarbonScanRDD(
+      ocRaw.sparkContext,
       model,
       buildCarbonPlan.getFilterExpression,
       kv,
@@ -578,78 +207,35 @@ case class CarbonTableScan(
   }
 
 
-  override def outputsUnsafeRows: Boolean = false
+  override def outputsUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
 
-  def doExecute(): RDD[InternalRow] = {
+  override def doExecute(): RDD[InternalRow] = {
     def toType(obj: Any): Any = {
       obj match {
         case s: String => UTF8String.fromString(s)
-        case avg: AbstractAvgAggregator =>
-          if (avg.isFirstTime) {
-            null
-          } else {
-            new GenericArrayData(avg.getAvgState.asInstanceOf[Array[Any]])
-          }
-        case c: CountAggregator => c.getLongValue
-        case s: SumDoubleAggregator => s.getDoubleValue
-        case s: SumBigDecimalAggregator => Decimal(s.getBigDecimalValue)
-        case s: SumLongAggregator => s.getLongValue
-        case m: MaxBigDecimalAggregator => Decimal(m.getBigDecimalValue)
-        case m: MaxLongAggregator => m.getLongValue
-        case m: MaxAggregator => toType(m.getValueObject)
-        case m: MinBigDecimalAggregator => Decimal(m.getBigDecimalValue)
-        case m: MinLongAggregator => m.getLongValue
-        case m: MinAggregator => toType(m.getValueObject)
-        case m: MeasureAggregator => toType(m.getValueObject)
         case _ => obj
       }
     }
+    val outUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
+    inputRdd.mapPartitions { iter =>
+      val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
+      new Iterator[InternalRow] {
+        override def hasNext: Boolean = iter.hasNext
 
-//    val unsafeProjection = UnsafeProjection.create(attributes.map(_.dataType).toArray)
-    // count(*) query executed in driver by querying from Btree
-    if (isCountQuery) {
-      val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-      val (carbonInputFormat: CarbonInputFormat[RowResult], job: Job) =
-        QueryPlanUtil.createCarbonInputFormat(absoluteTableIdentifier)
-      // get row count
-      val rowCount = carbonInputFormat.getRowCount(job)
-      val countAgg = new CountAggregator()
-      countAgg.setNewValue(rowCount)
-      sparkContext.parallelize(
-        Seq(new GenericMutableRow(Seq(countAgg.getLongValue).toArray.asInstanceOf[Array[Any]]))
-      )
-    } else {
-      // all the other queries are sent to executor
-      inputRdd.mapPartitions { iter =>
-        new Iterator[InternalRow] {
-//          val unsafeProjection = UnsafeProjection.create(attributes.map(_.dataType).toArray)
-          override def hasNext: Boolean = iter.hasNext
-
-          override def next(): InternalRow = {
-            new GenericMutableRow(iter.next()._1.getKey.map(toType))
+        override def next(): InternalRow =
+          if (outUnsafeRows) {
+            unsafeProjection(new GenericMutableRow(iter.next()._1.map(toType)))
+          } else {
+            new GenericMutableRow(iter.next()._1.map(toType))
           }
-        }
       }
     }
   }
 
-  /**
-   * return true if query is count queryUtils
- *
-   * @return
-   */
-  def isCountQuery: Boolean = {
-    if (buildCarbonPlan.isCountStarQuery() && null == buildCarbonPlan.getFilterExpression &&
-        buildCarbonPlan.getDimensions.size() < 1 && buildCarbonPlan.getMeasures.size() < 2 &&
-        buildCarbonPlan.getDimAggregatorInfos.size() < 1) {
-      true
-    } else {
-      false
-    }
-  }
-
   def output: Seq[Attribute] = {
-    attributes
+    attributesRaw
   }
+
 }
 
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala
deleted file mode 100644
index 2005300..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonRawOperators.scala
+++ /dev/null
@@ -1,332 +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
-
-import java.util.ArrayList
-
-import scala.collection.JavaConverters._
-import scala.collection.mutable.ArrayBuffer
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
-import org.apache.spark.sql.execution.LeafNode
-import org.apache.spark.sql.hive.CarbonMetastoreCatalog
-import org.apache.spark.sql.types.{DataType, Decimal}
-import org.apache.spark.unsafe.types.UTF8String
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier
-import org.carbondata.core.constants.CarbonCommonConstants
-import org.carbondata.core.util.CarbonProperties
-import org.carbondata.query.carbon.model._
-import org.carbondata.query.carbon.result.BatchRawResult
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper
-import org.carbondata.spark.{CarbonFilters, RawKey, RawKeyImpl, RawKeyVal, RawKeyValImpl}
-import org.carbondata.spark.rdd.CarbonRawQueryRDD
-
-
-case class CarbonRawTableScan(
-    var attributesRaw: Seq[Attribute],
-    relationRaw: CarbonRelation,
-    dimensionPredicatesRaw: Seq[Expression],
-    aggExprsRaw: Option[Seq[Expression]],
-    useBinaryAggregator: Boolean)(@transient val ocRaw: SQLContext) extends LeafNode
-{
-  val carbonTable = relationRaw.metaData.carbonTable
-  val selectedDims = scala.collection.mutable.MutableList[QueryDimension]()
-  val selectedMsrs = scala.collection.mutable.MutableList[QueryMeasure]()
-  @transient val carbonCatalog = ocRaw.catalog.asInstanceOf[CarbonMetastoreCatalog]
-
-  val attributesNeedToDecode = new java.util.HashSet[AttributeReference]()
-  val unprocessedExprs = new ArrayBuffer[Expression]()
-
-  val buildCarbonPlan: CarbonQueryPlan = {
-    val plan: CarbonQueryPlan = new CarbonQueryPlan(relationRaw.schemaName, relationRaw.tableName)
-
-    val dimensions = carbonTable.getDimensionByTableName(carbonTable.getFactTableName)
-    val measures = carbonTable.getMeasureByTableName(carbonTable.getFactTableName)
-    val dimAttr = new Array[Attribute](dimensions.size())
-    val msrAttr = new Array[Attribute](measures.size())
-    attributesRaw.foreach { attr =>
-      val carbonDimension =
-        carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
-      if(carbonDimension != null) {
-        dimAttr(dimensions.indexOf(carbonDimension)) = attr
-      } else {
-        val carbonMeasure =
-          carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name)
-        if(carbonMeasure != null) {
-          msrAttr(measures.indexOf(carbonMeasure)) = attr
-        }
-      }
-    }
-
-    attributesRaw = dimAttr.filter(f => f != null) ++ msrAttr.filter(f => f != null)
-
-    var queryOrder: Integer = 0
-    attributesRaw.foreach { attr =>
-        val carbonDimension =
-          carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
-        if (carbonDimension != null) {
-          val dim = new QueryDimension(attr.name)
-          dim.setQueryOrder(queryOrder)
-          queryOrder = queryOrder + 1
-          selectedDims += dim
-        } else {
-          val carbonMeasure =
-            carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name)
-          if (carbonMeasure != null) {
-            val m1 = new QueryMeasure(attr.name)
-            m1.setQueryOrder(queryOrder)
-            queryOrder = queryOrder + 1
-            selectedMsrs += m1
-          }
-        }
-      }
-    // Just find out that any aggregation functions are present on dimensions.
-    aggExprsRaw match {
-      case Some(aggExprs) =>
-        aggExprs.foreach {
-          case Alias(agg: AggregateExpression, name) =>
-            agg.collect {
-              case attr: AttributeReference =>
-                val dims = selectedDims.filter(m => m.getColumnName.equalsIgnoreCase(attr.name))
-                if(dims.nonEmpty) {
-                  plan.addAggDimAggInfo(dims.head.getColumnName,
-                    dims.head.getAggregateFunction,
-                    dims.head.getQueryOrder)
-                }
-            }
-          case _ =>
-        }
-      case _ =>
-    }
-
-    // Fill the selected dimensions & measures obtained from
-    // attributes to query plan  for detailed query
-    selectedDims.foreach(plan.addDimension)
-    selectedMsrs.foreach(plan.addMeasure)
-
-    plan.setSortedDimemsions(new ArrayList[QueryDimension])
-
-    plan.setRawDetailQuery(true)
-    plan.setOutLocationPath(
-      CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS))
-    plan.setQueryId(System.nanoTime() + "")
-    processFilterExpressions(plan)
-    plan
-  }
-
-  def processFilterExpressions(plan: CarbonQueryPlan) {
-    if (dimensionPredicatesRaw.nonEmpty) {
-      val expressionVal = CarbonFilters.processExpression(
-        dimensionPredicatesRaw,
-        attributesNeedToDecode,
-        unprocessedExprs,
-        carbonTable)
-      expressionVal match {
-        case Some(ce) =>
-          // adding dimension used in expression in querystats
-          plan.setFilterExpression(ce)
-        case _ =>
-      }
-    }
-    processExtraAttributes(plan)
-  }
-
-  private def processExtraAttributes(plan: CarbonQueryPlan) {
-    if (attributesNeedToDecode.size() > 0) {
-      val attributeOut = new ArrayBuffer[Attribute]() ++ attributesRaw
-
-      attributesNeedToDecode.asScala.map { attr =>
-        val dims = plan.getDimensions.asScala.filter(f => f.getColumnName.equals(attr.name))
-        val msrs = plan.getMeasures.asScala.filter(f => f.getColumnName.equals(attr.name))
-        var order = plan.getDimensions.size() + plan.getMeasures.size()
-        if (dims.isEmpty && msrs.isEmpty) {
-          val dimension = carbonTable.getDimensionByName(carbonTable.getFactTableName, attr.name)
-          if (dimension != null) {
-            val qDim = new QueryDimension(dimension.getColName)
-            qDim.setQueryOrder(order)
-            plan.addDimension(qDim)
-            attributeOut += attr
-            order += 1
-          } else {
-            val measure = carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name)
-            if (measure != null) {
-              val qMsr = new QueryMeasure(measure.getColName)
-              qMsr.setQueryOrder(order)
-              plan.addMeasure(qMsr)
-              order += 1
-              attributeOut += attr
-            }
-          }
-        }
-      }
-      attributesRaw = attributeOut
-    }
-  }
-
-
-  def inputRdd: CarbonRawQueryRDD[Array[Any], Any] = {
-
-    val conf = new Configuration()
-    val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
-    buildCarbonPlan.getDimAggregatorInfos.clear()
-    val model = QueryModel.createModel(
-      absoluteTableIdentifier, buildCarbonPlan, carbonTable)
-    val kv: RawKey[Array[Any], Any] = new RawKeyImpl()
-    // setting queryid
-    buildCarbonPlan.setQueryId(ocRaw.getConf("queryId", System.nanoTime() + ""))
-
-    val cubeCreationTime = carbonCatalog
-      .getCubeCreationTime(relationRaw.schemaName, relationRaw.tableName)
-    val schemaLastUpdatedTime = carbonCatalog
-      .getSchemaLastUpdatedTime(relationRaw.schemaName, relationRaw.tableName)
-    val big = new CarbonRawQueryRDD(
-      ocRaw.sparkContext,
-      model,
-      buildCarbonPlan.getFilterExpression,
-      kv,
-      conf,
-      cubeCreationTime,
-      schemaLastUpdatedTime,
-      carbonCatalog.storePath)
-    big
-  }
-
-
-  override def outputsUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
-
-  override def doExecute(): RDD[InternalRow] = {
-    def toType(obj: Any): Any = {
-      obj match {
-        case s: String => UTF8String.fromString(s)
-        case _ => obj
-      }
-    }
-    val outUnsafeRows: Boolean = attributesNeedToDecode.size() == 0
-    inputRdd.mapPartitions { iter =>
-      val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
-      new Iterator[InternalRow] {
-        override def hasNext: Boolean = iter.hasNext
-
-        override def next(): InternalRow =
-          if (outUnsafeRows) {
-            unsafeProjection(new GenericMutableRow(iter.next()._1.map(toType)))
-          } else {
-            new GenericMutableRow(iter.next()._1.map(toType))
-          }
-      }
-    }
-  }
-
-  def output: Seq[Attribute] = {
-    attributesRaw
-  }
-
-}
-
-class CarbonRawMutableRow(values: Array[Array[Object]],
-    val schema: QuerySchemaInfo) extends GenericMutableRow(values.asInstanceOf[Array[Any]]) {
-
-  val dimsLen = schema.getQueryDimensions.length - 1
-  val order = schema.getQueryOrder
-  var counter = 0
-  val size = {
-    if (values.nonEmpty) {
-      values.head.length
-    } else {
-      0
-    }
-  }
-
-  def getKey: ByteArrayWrapper = values.head(counter).asInstanceOf[ByteArrayWrapper]
-
-  def parseKey(key: ByteArrayWrapper, aggData: Array[Object], order: Array[Int]): Array[Object] = {
-    BatchRawResult.parseData(key, aggData, schema, order)
-  }
-
-  def hasNext: Boolean = {
-    counter < size
-  }
-
-  def next(): Unit = {
-    counter += 1
-  }
-
-  override def numFields: Int = dimsLen + schema.getQueryMeasures.length
-
-  override def anyNull: Boolean = true
-
-  override def get(ordinal: Int, dataType: DataType): AnyRef = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[AnyRef]
-  }
-
-  override def getUTF8String(ordinal: Int): UTF8String = {
-    UTF8String
-      .fromString(values(
-        order(ordinal) - dimsLen)(counter)
-        .asInstanceOf[String])
-  }
-
-  override def getDouble(ordinal: Int): Double = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[Double]
-  }
-
-  override def getFloat(ordinal: Int): Float = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[Float]
-  }
-
-  override def getLong(ordinal: Int): Long = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[Long]
-  }
-
-  override def getByte(ordinal: Int): Byte = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[Byte]
-  }
-
-  override def getDecimal(ordinal: Int,
-      precision: Int,
-      scale: Int): Decimal = {
-    values(order(ordinal) - dimsLen)(counter).asInstanceOf[Decimal]
-  }
-
-  override def getBoolean(ordinal: Int): Boolean = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[Boolean]
-  }
-
-  override def getShort(ordinal: Int): Short = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[Short]
-  }
-
-  override def getInt(ordinal: Int): Int = {
-    values(order(ordinal) - dimsLen)(counter)
-      .asInstanceOf[Int]
-  }
-
-  override def isNullAt(ordinal: Int): Boolean = values(order(ordinal) - dimsLen)(counter) == null
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/execution/joins/CarbonJoins.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/joins/CarbonJoins.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/joins/CarbonJoins.scala
deleted file mode 100644
index 79ba157..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/joins/CarbonJoins.scala
+++ /dev/null
@@ -1,140 +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.joins
-
-import scala.concurrent._
-import scala.concurrent.duration._
-import scala.Array.canBuildFrom
-
-import org.apache.spark.{InternalAccumulator, TaskContext}
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.CarbonTableScan
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.{BindReferences, Expression, Literal}
-import org.apache.spark.sql.execution.{BinaryNode, SparkPlan, SQLExecution}
-import org.apache.spark.sql.execution.metric.SQLMetrics
-import org.apache.spark.unsafe.types.UTF8String
-import org.apache.spark.util.ThreadUtils
-
-case class BroadCastFilterPushJoin(
-    leftKeys: Seq[Expression],
-    rightKeys: Seq[Expression],
-    buildSide: BuildSide,
-    left: SparkPlan,
-    right: SparkPlan,
-    condition: Option[Expression]) extends BinaryNode with HashJoin {
-
-  override private[sql] lazy val metrics = Map(
-    "numLeftRows" -> SQLMetrics.createLongMetric(sparkContext, "number of left rows"),
-    "numRightRows" -> SQLMetrics.createLongMetric(sparkContext, "number of right rows"),
-    "numOutputRows" -> SQLMetrics.createLongMetric(sparkContext, "number of output rows"))
-
-  val timeout: Duration = {
-    val timeoutValue = sqlContext.conf.broadcastTimeout
-    if (timeoutValue < 0) {
-      Duration.Inf
-    } else {
-      timeoutValue.seconds
-    }
-  }
-  private lazy val (input: Array[InternalRow], inputCopy: Array[InternalRow]) = {
-    val numBuildRows = buildSide match {
-      case BuildLeft => longMetric("numLeftRows")
-      case BuildRight => longMetric("numRightRows")
-    }
-    val buildPlanOutput = buildPlan.execute()
-    val input: Array[InternalRow] = buildPlanOutput.map(_.copy()).collect()
-    val inputCopy: Array[InternalRow] = buildPlanOutput.map(_.copy()).collect()
-    (input, inputCopy)
-  }
-  // Use lazy so that we won't do broadcast when calling explain but still cache the broadcast value
-  // for the same query.
-  @transient
-  private lazy val broadcastFuture = {
-    // broadcastFuture is used in "doExecute". Therefore we can get the execution id correctly here.
-    val executionId = sparkContext.getLocalProperty(SQLExecution.EXECUTION_ID_KEY)
-    future {
-      // This will run in another thread. Set the execution id so that we can connect these jobs
-      // with the correct execution.
-      SQLExecution.withExecutionId(sparkContext, executionId) {
-        // The following line doesn't run in a job so we cannot track the metric value. However, we
-        // have already tracked it in the above lines. So here we can use
-        // `SQLMetrics.nullLongMetric` to ignore it.
-        val hashed = HashedRelation(
-          input.iterator, SQLMetrics.nullLongMetric, buildSideKeyGenerator, input.size)
-        sparkContext.broadcast(hashed)
-      }
-    }(BroadCastFilterPushJoin.broadcastHashJoinExecutionContext)
-  }
-
-  override def doExecute(): RDD[InternalRow] = {
-
-    val numOutputRows = longMetric("numOutputRows")
-    val (numBuildRows, numStreamedRows) = buildSide match {
-      case BuildLeft => (longMetric("numLeftRows"), longMetric("numRightRows"))
-      case BuildRight => (longMetric("numRightRows"), longMetric("numLeftRows"))
-    }
-
-    val keys = buildKeys.map { a =>
-      BindReferences.bindReference(a, buildPlan.output)
-    }.toArray
-    val filters = keys.map {
-      k =>
-        inputCopy.map(
-          r => {
-            val curr = k.eval(r)
-            if (curr.isInstanceOf[UTF8String]) {
-              Literal(curr.toString).asInstanceOf[Expression]
-            } else {
-              Literal(curr).asInstanceOf[Expression]
-            }
-          })
-    }
-    val carbonScan = buildSide match {
-      case BuildLeft => right
-      case BuildRight => left
-    }
-
-    val cubeScan = carbonScan.collectFirst { case a: CarbonTableScan => a }
-    if (cubeScan.isDefined) {
-      cubeScan.get.addPushdownFilters(streamedKeys, filters, condition)
-    }
-
-    val streamedPlanOutput = streamedPlan.execute()
-    // scalastyle:off
-    val broadcastRelation = Await.result(broadcastFuture, timeout)
-    // scalastyle:on
-    streamedPlanOutput.mapPartitions { streamedIter =>
-      val hashedRelation = broadcastRelation.value
-      hashedRelation match {
-        case unsafe: UnsafeHashedRelation =>
-          TaskContext.get().internalMetricsToAccumulators(
-            InternalAccumulator.PEAK_EXECUTION_MEMORY).add(unsafe.getUnsafeSize)
-        case _ =>
-      }
-      hashJoin(streamedIter, numStreamedRows, hashedRelation, numOutputRows)
-    }
-
-  }
-}
-
-object BroadCastFilterPushJoin {
-
-  private[joins] val broadcastHashJoinExecutionContext = ExecutionContext.fromExecutorService(
-    ThreadUtils.newDaemonCachedThreadPool("filterpushhash-join", 128))
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.scala
deleted file mode 100644
index c01a937..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonRawStrategies.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.spark.sql.hive
-
-import scala.collection.JavaConverters._
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.expressions
-import org.apache.spark.sql.catalyst.expressions.{AttributeSet, _}
-import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, QueryPlanner}
-import org.apache.spark.sql.catalyst.plans.logical.{Filter => LogicalFilter, LogicalPlan}
-import org.apache.spark.sql.execution.{Filter, Project, SparkPlan}
-import org.apache.spark.sql.execution.datasources.LogicalRelation
-import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
-
-import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
-
-class CarbonRawStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
-
-  override def strategies: Seq[Strategy] = getStrategies
-
-  val LOGGER = LogServiceFactory.getLogService("CarbonRawStrategies")
-
-  def getStrategies: Seq[Strategy] = {
-    val total = sqlContext.planner.strategies :+ CarbonRawTableScans
-    total
-  }
-
-  /**
-   * Carbon strategies for Carbon cube scanning
-   */
-  private[sql] object CarbonRawTableScans extends Strategy {
-
-    def apply(plan: LogicalPlan): Seq[SparkPlan] = {
-      plan match {
-        case PhysicalOperation(projectList, predicates,
-        l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)) =>
-          if (isStarQuery(plan)) {
-            carbonRawScanForStarQuery(projectList, predicates, carbonRelation, l)(sqlContext) :: Nil
-          } else {
-            carbonRawScan(projectList,
-              predicates,
-              carbonRelation,
-              l,
-              None,
-              detailQuery = true,
-              useBinaryAggregation = false)(sqlContext)._1 :: Nil
-          }
-        case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
-          CarbonDictionaryDecoder(relations,
-            profile,
-            aliasMap,
-            planLater(child))(sqlContext) :: Nil
-        case _ =>
-          Nil
-      }
-    }
-
-    /**
-     * Create carbon scan
-     */
-    private def carbonRawScan(projectList: Seq[NamedExpression],
-        predicates: Seq[Expression],
-        relation: CarbonDatasourceRelation,
-        logicalRelation: LogicalRelation,
-        groupExprs: Option[Seq[Expression]],
-        detailQuery: Boolean,
-        useBinaryAggregation: Boolean)(sc: SQLContext): (SparkPlan, Boolean) = {
-
-      val tableName: String =
-        relation.carbonRelation.metaData.carbonTable.getFactTableName.toLowerCase
-      // Check out any expressions are there in project list. if they are present then we need to
-      // decode them as well.
-      val projectSet = AttributeSet(projectList.flatMap(_.references))
-      val scan = CarbonRawTableScan(projectSet.toSeq,
-        relation.carbonRelation,
-        predicates,
-        groupExprs,
-        useBinaryAggregation)(sqlContext)
-      val dimAggrsPresence: Boolean = scan.buildCarbonPlan.getDimAggregatorInfos.size() > 0
-      projectList.map {
-        case attr: AttributeReference =>
-        case Alias(attr: AttributeReference, _) =>
-        case others =>
-          others.references
-            .map(f => scan.attributesNeedToDecode.add(f.asInstanceOf[AttributeReference]))
-      }
-      if (!detailQuery) {
-        if (scan.attributesNeedToDecode.size > 0) {
-          val decoder = getCarbonDecoder(logicalRelation,
-            sc,
-            tableName,
-            scan.attributesNeedToDecode.asScala.toSeq,
-            scan)
-          if (scan.unprocessedExprs.nonEmpty) {
-            val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
-            (Project(projectList, filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)), true)
-          } else {
-            (Project(projectList, decoder), true)
-          }
-        } else {
-          (scan, dimAggrsPresence)
-        }
-      } else {
-        if (scan.attributesNeedToDecode.size() > 0) {
-          val decoder = getCarbonDecoder(logicalRelation,
-            sc,
-            tableName,
-            scan.attributesNeedToDecode.asScala.toSeq,
-            scan)
-          if (scan.unprocessedExprs.nonEmpty) {
-            val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
-            (Project(projectList, filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)), true)
-          } else {
-            (Project(projectList, decoder), true)
-          }
-        } else {
-          (Project(projectList, scan), dimAggrsPresence)
-        }
-      }
-    }
-
-    /**
-     * Create carbon scan for star query
-     */
-    private def carbonRawScanForStarQuery(projectList: Seq[NamedExpression],
-        predicates: Seq[Expression],
-        relation: CarbonDatasourceRelation,
-        logicalRelation: LogicalRelation)(sc: SQLContext): SparkPlan = {
-
-      val tableName: String =
-        relation.carbonRelation.metaData.carbonTable.getFactTableName.toLowerCase
-      // Check out any expressions are there in project list. if they are present then we need to
-      // decode them as well.
-      val projectExprsNeedToDecode = new java.util.HashSet[Attribute]()
-      val scan = CarbonRawTableScan(projectList.map(_.toAttribute),
-        relation.carbonRelation,
-        predicates,
-        None,
-        useBinaryAggregator = false)(sqlContext)
-      projectExprsNeedToDecode.addAll(scan.attributesNeedToDecode)
-      if (projectExprsNeedToDecode.size() > 0) {
-        val decoder = getCarbonDecoder(logicalRelation,
-          sc,
-          tableName,
-          projectExprsNeedToDecode.asScala.toSeq,
-          scan)
-        if (scan.unprocessedExprs.nonEmpty) {
-          val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
-          filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)
-        } else {
-          decoder
-        }
-      } else {
-        scan
-      }
-    }
-
-    def getCarbonDecoder(logicalRelation: LogicalRelation,
-        sc: SQLContext,
-        tableName: String,
-        projectExprsNeedToDecode: Seq[Attribute],
-        scan: CarbonRawTableScan): CarbonDictionaryDecoder = {
-      val relation = CarbonDecoderRelation(logicalRelation.attributeMap,
-        logicalRelation.relation.asInstanceOf[CarbonDatasourceRelation])
-      val attrs = projectExprsNeedToDecode.map { attr =>
-        val newAttr = AttributeReference(attr.name,
-          attr.dataType,
-          attr.nullable,
-          attr.metadata)(attr.exprId, Seq(tableName))
-        relation.addAttribute(newAttr)
-        newAttr
-      }
-      CarbonDictionaryDecoder(Seq(relation), IncludeProfile(attrs),
-        CarbonAliasDecoderRelation(), scan)(sc)
-    }
-
-    private def isStarQuery(plan: LogicalPlan) = {
-      plan match {
-        case LogicalFilter(condition,
-        LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)) => true
-        case LogicalRelation(carbonRelation: CarbonDatasourceRelation, _) => true
-        case _ => false
-      }
-    }
-
-    private def isGroupByPresentOnMeasures(groupingExpressions: Seq[Expression],
-        carbonTable: CarbonTable): Boolean = {
-      groupingExpressions.map { g =>
-       g.collect {
-         case attr: AttributeReference
-           if carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) != null =>
-           return true
-       }
-      }
-      false
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
new file mode 100644
index 0000000..edfaa90
--- /dev/null
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
@@ -0,0 +1,42 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.spark.sql.hive
+
+import org.apache.spark.sql.CarbonSqlParser
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+
+import org.carbondata.spark.exception.MalformedCarbonCommandException
+
+private[spark] class CarbonSQLDialect(context: HiveContext) extends HiveQLDialect(context) {
+
+  @transient
+  protected val sqlParser = new CarbonSqlParser
+
+  override def parse(sqlText: String): LogicalPlan = {
+
+    try {
+      sqlParser.parse(sqlText)
+    } catch {
+      // MalformedCarbonCommandException need to throw directly
+      // because hive can no parse carbon command
+      case ce: MalformedCarbonCommandException =>
+        throw ce
+      case _ => super.parse(sqlText)
+    }
+  }
+}


[19/56] [abbrv] incubator-carbondata git commit: Refactored core package and fixed all testcases (#684)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.java
deleted file mode 100644
index 2f44bca..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.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.carbondata.query.aggregator.impl.distinct;
-
-import java.math.BigDecimal;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctCountBigDecimalAggregatorObjectSet
-    extends AbstractDistinctCountAggregatorObjectSet {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-    }
-  }
-
-  private void agg(Set<Object> set2) {
-    valueSetForObj.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountBigDecimalAggregatorObjectSet distinctCountBigDecimalAggregatorObjectSet =
-        (DistinctCountBigDecimalAggregatorObjectSet) aggregator;
-    agg(distinctCountBigDecimalAggregatorObjectSet.valueSetForObj);
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountBigDecimalAggregatorObjectSet aggregator =
-        new DistinctCountBigDecimalAggregatorObjectSet();
-    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    BigDecimal valueSetForObjSize = getBigDecimalValue();
-    BigDecimal otherVal = measureAggr.getBigDecimalValue();
-    return valueSetForObjSize.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof DistinctCountBigDecimalAggregatorObjectSet)) {
-      return false;
-    }
-    DistinctCountBigDecimalAggregatorObjectSet o = (DistinctCountBigDecimalAggregatorObjectSet) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DistinctCountBigDecimalAggregatorObjectSet();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java
deleted file mode 100644
index c4f7216..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java
+++ /dev/null
@@ -1,89 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctCountLongAggregatorObjectSet extends AbstractDistinctCountAggregatorObjectSet {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-    }
-  }
-
-  private void agg(Set<Object> set2) {
-    valueSetForObj.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountLongAggregatorObjectSet distinctCountAggregator =
-        (DistinctCountLongAggregatorObjectSet) aggregator;
-    agg(distinctCountAggregator.valueSetForObj);
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountLongAggregatorObjectSet aggregator = new DistinctCountLongAggregatorObjectSet();
-    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    long valueSetForObjSize = getLongValue();
-    long otherVal = measureAggr.getLongValue();
-    if (valueSetForObjSize > otherVal) {
-      return 1;
-    }
-    if (valueSetForObjSize < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof DistinctCountLongAggregatorObjectSet)) {
-      return false;
-    }
-    DistinctCountLongAggregatorObjectSet o = (DistinctCountLongAggregatorObjectSet) obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DistinctCountLongAggregatorObjectSet();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java
deleted file mode 100644
index e3d4623..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java
+++ /dev/null
@@ -1,165 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctStringCountAggregator implements MeasureAggregator {
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  private Set<String> valueSetForStr;
-
-  public DistinctStringCountAggregator() {
-    this.valueSetForStr = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  public void agg(double newVal) {
-  }
-
-  public void agg(String newVal) {
-    this.valueSetForStr.add(newVal);
-  }
-
-  private void agg(Set<String> set2) {
-    this.valueSetForStr.addAll(set2);
-  }
-
-  public void merge(MeasureAggregator aggregator) {
-    DistinctStringCountAggregator distinctCountAggregator =
-        (DistinctStringCountAggregator) aggregator;
-    agg(distinctCountAggregator.valueSetForStr);
-  }
-
-  public Double getDoubleValue() {
-    return (double) this.valueSetForStr.size();
-  }
-
-  public Long getLongValue() {
-    return (long) this.valueSetForStr.size();
-  }
-
-  public BigDecimal getBigDecimalValue() {
-    return new BigDecimal(this.valueSetForStr.size());
-  }
-
-  public Object getValueObject() {
-    return Integer.valueOf(this.valueSetForStr.size());
-  }
-
-  public void setNewValue(Object newValue) {
-  }
-
-  public boolean isFirstTime() {
-    return false;
-  }
-
-  public void writeData(DataOutput output) throws IOException {
-    int length = this.valueSetForStr.size() * 8;
-    ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4);
-    byteBuffer.putInt(length);
-    for (String val : this.valueSetForStr) {
-      byte[] b = val.getBytes(Charset.defaultCharset());
-      byteBuffer.putInt(b.length);
-      byteBuffer.put(b);
-    }
-    byteBuffer.flip();
-    output.write(byteBuffer.array());
-  }
-
-  public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-    length /= 8;
-    this.valueSetForStr = new HashSet<String>(length + 1, 1.0F);
-    for (int i = 0; i < length; i++) {
-      byte[] b = new byte[inPut.readInt()];
-      inPut.readFully(b);
-      this.valueSetForStr.add(new String(b, Charset.defaultCharset()));
-    }
-  }
-
-  public MeasureAggregator getCopy() {
-    DistinctStringCountAggregator aggregator = new DistinctStringCountAggregator();
-    aggregator.valueSetForStr = new HashSet<String>(this.valueSetForStr);
-    return aggregator;
-  }
-
-  public int compareTo(MeasureAggregator o) {
-    double val = getDoubleValue();
-    double otherVal = o.getDoubleValue();
-    if (val > otherVal) {
-      return 1;
-    }
-    if (val < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof DistinctStringCountAggregator)) {
-      return false;
-    }
-    DistinctStringCountAggregator o = (DistinctStringCountAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public void agg(Object newVal) {
-    this.valueSetForStr.add((String) newVal);
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-  }
-
-  @Override public byte[] getByteArray() {
-    return null;
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  public String toString() {
-    return valueSetForStr.size() + "";
-  }
-
-  @Override public void merge(byte[] value) {
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new DistinctStringCountAggregator();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java
deleted file mode 100644
index 6a59ec9..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java
+++ /dev/null
@@ -1,233 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-/**
- * The sum distinct aggregator
- * Ex:
- * ID NAME Sales
- * 1 a 200
- * 2 a 100
- * 3 a 200
- * select sum(distinct sales) # would result 300
- */
-public class SumDistinctBigDecimalAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 6313463368629960155L;
-
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten the
-   * Aggregators. There is no aggregation expected after setting this value.
-   */
-  private BigDecimal computedFixedValue;
-
-  /**
-   *
-   */
-  private Set<BigDecimal> valueSet;
-
-  public SumDistinctBigDecimalAggregator() {
-    valueSet = new HashSet<BigDecimal>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * Distinct Aggregate function which update the Distinct set
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSet.add(
-        newVal instanceof BigDecimal ? (BigDecimal) newVal : new BigDecimal(newVal.toString()));
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    Iterator<BigDecimal> iterator = valueSet.iterator();
-    ByteBuffer buffer =
-        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    while (iterator.hasNext()) {
-      byte[] bytes = DataTypeUtil.bigDecimalToByte(iterator.next());
-      buffer.putInt(bytes.length);
-      buffer.put(bytes);
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(Set<BigDecimal> set2) {
-    valueSet.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    SumDistinctBigDecimalAggregator distinctAggregator =
-        (SumDistinctBigDecimalAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(distinctAggregator.valueSet);
-      firstTime = false;
-    }
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    if (computedFixedValue == null) {
-      BigDecimal result = new BigDecimal(0);
-      for (BigDecimal aValue : valueSet) {
-        result = result.add(aValue);
-      }
-      return result;
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Object getValueObject() {
-    return getBigDecimalValue();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (BigDecimal) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void writeData(DataOutput dataOutput) throws IOException {
-    if (computedFixedValue != null) {
-      byte[] bytes = DataTypeUtil.bigDecimalToByte(computedFixedValue);
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 4 + bytes.length);
-      byteBuffer.putInt(-1);
-      byteBuffer.putInt(bytes.length);
-      byteBuffer.put(bytes);
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    } else {
-      int length = valueSet.size() * 8 + valueSet.size() * 4;
-      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
-      byteBuffer.putInt(length);
-      for (BigDecimal val : valueSet) {
-        byte[] bytes =
-            val.toString().getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
-        byteBuffer.putInt(-1);
-        byteBuffer.putInt(bytes.length);
-        byteBuffer.put(bytes);
-      }
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-
-    if (length == -1) {
-      computedFixedValue = new BigDecimal(inPut.readUTF());
-      valueSet = null;
-    } else {
-      length = length / 8;
-      valueSet = new HashSet<BigDecimal>(length + 1, 1.0f);
-      for (int i = 0; i < length; i++) {
-        valueSet.add(new BigDecimal(inPut.readUTF()));
-      }
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      byte[] valueByte = new byte[buffer.getInt()];
-      buffer.get(valueByte);
-      BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-      agg(valueBigDecimal);
-    }
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      return valueSet.size() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDistinctBigDecimalAggregator aggregator = new SumDistinctBigDecimalAggregator();
-    aggregator.valueSet = new HashSet<BigDecimal>(valueSet);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator msr) {
-    BigDecimal msrValObj = getBigDecimalValue();
-    BigDecimal otherVal = msr.getBigDecimalValue();
-
-    return msrValObj.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumDistinctBigDecimalAggregator)) {
-      return false;
-    }
-    SumDistinctBigDecimalAggregator o = (SumDistinctBigDecimalAggregator) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumDistinctBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java
deleted file mode 100644
index 0229f24..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java
+++ /dev/null
@@ -1,231 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-/**
- * The sum distinct aggregator
- * Ex:
- * ID NAME Sales
- * 1 a 200
- * 2 a 100
- * 3 a 200
- * select sum(distinct sales) # would result 300
- */
-
-public class SumDistinctDoubleAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 6313463368629960155L;
-
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten the
-   * Aggregators. There is no aggregation expected after setting this value.
-   */
-  private Double computedFixedValue;
-
-  /**
-   *
-   */
-  private Set<Double> valueSet;
-
-  public SumDistinctDoubleAggregator() {
-    valueSet = new HashSet<Double>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-    valueSet.add(newVal);
-    firstTime = false;
-  }
-
-  /**
-   * Distinct Aggregate function which update the Distinct set
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSet.add(newVal instanceof Double ? (Double) newVal : new Double(newVal.toString()));
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    Iterator<Double> iterator = valueSet.iterator();
-    ByteBuffer buffer =
-        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    while (iterator.hasNext()) {
-      buffer.putDouble(iterator.next());
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(Set<Double> set2) {
-    valueSet.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    SumDistinctDoubleAggregator distinctAggregator = (SumDistinctDoubleAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(distinctAggregator.valueSet);
-      firstTime = false;
-    }
-  }
-
-  @Override public Double getDoubleValue() {
-    if (computedFixedValue == null) {
-      double result = 0;
-      for (Double aValue : valueSet) {
-        result += aValue;
-      }
-      return result;
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Object getValueObject() {
-    return getDoubleValue();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (Double) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void writeData(DataOutput dataOutput) throws IOException {
-    if (computedFixedValue != null) {
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
-      byteBuffer.putInt(-1);
-      byteBuffer.putDouble(computedFixedValue);
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    } else {
-      int length = valueSet.size() * 8;
-      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
-      byteBuffer.putInt(length);
-      for (double val : valueSet) {
-        byteBuffer.putDouble(val);
-      }
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-
-    if (length == -1) {
-      computedFixedValue = inPut.readDouble();
-      valueSet = null;
-    } else {
-      length = length / 8;
-      valueSet = new HashSet<Double>(length + 1, 1.0f);
-      for (int i = 0; i < length; i++) {
-        valueSet.add(inPut.readDouble());
-      }
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      agg(buffer.getDouble());
-    }
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      return valueSet.size() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDistinctDoubleAggregator aggregator = new SumDistinctDoubleAggregator();
-    aggregator.valueSet = new HashSet<Double>(valueSet);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator msr) {
-    double msrValObj = getDoubleValue();
-    double otherVal = msr.getDoubleValue();
-    if (msrValObj > otherVal) {
-      return 1;
-    }
-    if (msrValObj < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumDistinctDoubleAggregator)) {
-      return false;
-    }
-    SumDistinctDoubleAggregator o = (SumDistinctDoubleAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumDistinctDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java
deleted file mode 100644
index d57c34b..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java
+++ /dev/null
@@ -1,221 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-/**
- * The sum distinct aggregator
- * Ex:
- * ID NAME Sales
- * 1 a 200
- * 2 a 100
- * 3 a 200
- * select sum(distinct sales) # would result 300
- */
-
-public class SumDistinctLongAggregator extends AbstractMeasureAggregatorBasic {
-
-  private static final long serialVersionUID = 6313463368629960155L;
-
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten the
-   * Aggregators. There is no aggregation expected after setting this value.
-   */
-  private Long computedFixedValue;
-
-  /**
-   *
-   */
-  private Set<Long> valueSet;
-
-  public SumDistinctLongAggregator() {
-    valueSet = new HashSet<Long>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * Distinct Aggregate function which update the Distinct set
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSet.add(newVal instanceof Long ? (Long) newVal : Long.valueOf(newVal.toString()));
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    Iterator<Long> iterator = valueSet.iterator();
-    ByteBuffer buffer =
-        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    while (iterator.hasNext()) {
-      buffer.putLong(iterator.next());
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(Set<Long> set2) {
-    valueSet.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    SumDistinctLongAggregator distinctAggregator = (SumDistinctLongAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(distinctAggregator.valueSet);
-      firstTime = false;
-    }
-  }
-
-  @Override public Long getLongValue() {
-    if (computedFixedValue == null) {
-      long result = 0;
-      for (Long aValue : valueSet) {
-        result += aValue;
-      }
-      return result;
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Object getValueObject() {
-    return getLongValue();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (Long) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void writeData(DataOutput dataOutput) throws IOException {
-    if (computedFixedValue != null) {
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
-      byteBuffer.putInt(-1);
-      byteBuffer.putLong(computedFixedValue);
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    } else {
-      int length = valueSet.size() * 8;
-      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
-      byteBuffer.putInt(length);
-      for (long val : valueSet) {
-        byteBuffer.putLong(val);
-      }
-      byteBuffer.flip();
-      dataOutput.write(byteBuffer.array());
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-
-    if (length == -1) {
-      computedFixedValue = inPut.readLong();
-      valueSet = null;
-    } else {
-      length = length / 8;
-      valueSet = new HashSet<Long>(length + 1, 1.0f);
-      for (int i = 0; i < length; i++) {
-        valueSet.add(inPut.readLong());
-      }
-    }
-
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      agg(buffer.getLong());
-    }
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      return valueSet.size() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDistinctLongAggregator aggregator = new SumDistinctLongAggregator();
-    aggregator.valueSet = new HashSet<Long>(valueSet);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator msr) {
-    long msrValObj = getLongValue();
-    long otherVal = msr.getLongValue();
-    if (msrValObj > otherVal) {
-      return 1;
-    }
-    if (msrValObj < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumDistinctLongAggregator)) {
-      return false;
-    }
-    SumDistinctLongAggregator o = (SumDistinctLongAggregator) obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumDistinctLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java
deleted file mode 100644
index f423085..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java
+++ /dev/null
@@ -1,70 +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.carbondata.query.aggregator.impl.dummy;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-/**
- * AbstractMeasureAggregatorDummy
- * Used for custom Carbon Aggregator dummy
- */
-public abstract class AbstractMeasureAggregatorDummy extends AbstractMeasureAggregatorBasic {
-  private static final long serialVersionUID = 1L;
-
-  @Override public int compareTo(MeasureAggregator o) {
-    if (equals(o)) {
-      return 0;
-    }
-    return -1;
-  }
-
-  @Override public boolean equals(Object arg0) {
-    return super.equals(arg0);
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode();
-  }
-
-  @Override public byte[] getByteArray() {
-    return null;
-  }
-
-  @Override public void merge(MeasureAggregator aggregator) {
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    return null;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-  }
-
-  @Override public void merge(byte[] value) {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java
deleted file mode 100644
index 8a33fe1..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java
+++ /dev/null
@@ -1,63 +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.carbondata.query.aggregator.impl.dummy;
-
-import java.math.BigDecimal;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DummyBigDecimalAggregator extends AbstractMeasureAggregatorDummy {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * aggregate value
-   */
-  private BigDecimal aggVal;
-
-  @Override public void agg(Object newVal) {
-    aggVal = (BigDecimal) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return aggVal;
-  }
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (BigDecimal) newValue;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new DummyBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java
deleted file mode 100644
index cd4fe56..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java
+++ /dev/null
@@ -1,64 +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.carbondata.query.aggregator.impl.dummy;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DummyDoubleAggregator extends AbstractMeasureAggregatorDummy {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * aggregate value
-   */
-  private double aggVal;
-
-  @Override public void agg(double newVal) {
-    aggVal = newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(Object newVal) {
-    aggVal = (Double) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  @Override public Double getDoubleValue() {
-    return aggVal;
-  }
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Double) newValue;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DummyDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java
deleted file mode 100644
index 4131895..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java
+++ /dev/null
@@ -1,59 +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.carbondata.query.aggregator.impl.dummy;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DummyLongAggregator extends AbstractMeasureAggregatorDummy {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * aggregate value
-   */
-  private long aggVal;
-
-  @Override public void agg(Object newVal) {
-    aggVal = (Long) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  @Override public Long getLongValue() {
-    return aggVal;
-  }
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Long) newValue;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DummyLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java
deleted file mode 100644
index 27b1876..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.aggregator.impl.max;
-
-import java.io.ByteArrayInputStream;
-import java.io.ObjectInput;
-import java.io.ObjectInputStream;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorMaxMin;
-
-public abstract class AbstractMaxAggregator extends AbstractMeasureAggregatorMaxMin {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractMaxAggregator.class.getName());
-
-  protected void internalAgg(Object value) {
-    if (value instanceof Comparable) {
-      @SuppressWarnings("unchecked") Comparable<Object> newValue = ((Comparable<Object>) value);
-      aggVal = (aggVal == null || aggVal.compareTo(newValue) < 0) ? newValue : aggVal;
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteArrayInputStream bytesInputStream = null;
-    ObjectInput in = null;
-    try {
-      bytesInputStream = new ByteArrayInputStream(value);
-      in = new ObjectInputStream(bytesInputStream);
-      Object newVal = (Comparable<Object>) in.readObject();
-      internalAgg(newVal);
-      firstTime = false;
-    } catch (Exception e) {
-      LOGGER.error(e, "Problem while merging byte array in maxAggregator: " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(bytesInputStream);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.java
deleted file mode 100644
index d0d8af0..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.java
+++ /dev/null
@@ -1,68 +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.carbondata.query.aggregator.impl.max;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description :
- * It will return max of values
- */
-public class MaxAggregator extends AbstractMaxAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -5850218739083899419L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the max aggregate value if aggregator
-   * passed as an argument will have value greater than aggVal
-   *
-   * @param aggregator MaxAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(maxAggregator.aggVal);
-      firstTime = false;
-    }
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MaxAggregator aggregator = new MaxAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MaxAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java
deleted file mode 100644
index c4149c6..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java
+++ /dev/null
@@ -1,67 +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.carbondata.query.aggregator.impl.max;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description :
- * It will return max of values
- */
-public class MaxBigDecimalAggregator extends AbstractMaxAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -5850218739083899419L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the max aggregate value if aggregator
-   * passed as an argument will have value greater than aggVal
-   *
-   * @param aggregator MaxAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(maxAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MaxAggregator aggregator = new MaxAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MaxBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java
deleted file mode 100644
index 55b3be5..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java
+++ /dev/null
@@ -1,67 +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.carbondata.query.aggregator.impl.max;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description :
- * It will return max of values
- */
-public class MaxLongAggregator extends AbstractMaxAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -5850218739083899419L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the max aggregate value if aggregator
-   * passed as an argument will have value greater than aggVal
-   *
-   * @param aggregator MaxAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(maxAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MaxAggregator aggregator = new MaxAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MaxLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java
deleted file mode 100644
index 77aa163..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.aggregator.impl.min;
-
-import java.io.ByteArrayInputStream;
-import java.io.ObjectInput;
-import java.io.ObjectInputStream;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorMaxMin;
-
-public abstract class AbstractMinAggregator extends AbstractMeasureAggregatorMaxMin {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractMinAggregator.class.getName());
-
-  protected void internalAgg(Object value) {
-    if (value instanceof Comparable) {
-      @SuppressWarnings("unchecked") Comparable<Object> newValue = ((Comparable<Object>) value);
-      aggVal = (aggVal == null || aggVal.compareTo(newValue) > 0) ? newValue : aggVal;
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteArrayInputStream bis = null;
-    ObjectInput objectInput = null;
-    try {
-      bis = new ByteArrayInputStream(value);
-      objectInput = new ObjectInputStream(bis);
-      Object newVal = (Comparable<Object>) objectInput.readObject();
-      internalAgg(newVal);
-      firstTime = false;
-    } catch (Exception e) {
-      LOGGER.error(e, "Problem while merging byte array in minAggregator: " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(bis);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java
deleted file mode 100644
index 4bb3d73..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.aggregator.impl.min;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return min of values
- */
-public class MinAggregator extends AbstractMinAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8077547753784906280L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the min aggregate value if aggregator
-   * passed as an argument will have value less than aggVal
-   *
-   * @param aggregator MinAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MinAggregator minAggregator = (MinAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(minAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MinAggregator aggregator = new MinAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MinAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java
deleted file mode 100644
index 7347dc1..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.aggregator.impl.min;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return min of values
- */
-public class MinBigDecimalAggregator extends AbstractMinAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8077547753784906280L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the min aggregate value if aggregator
-   * passed as an argument will have value less than aggVal
-   *
-   * @param aggregator MinAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MinAggregator minAggregator = (MinAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(minAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MinAggregator aggregator = new MinAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MinBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java
deleted file mode 100644
index af614c9..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java
+++ /dev/null
@@ -1,66 +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.carbondata.query.aggregator.impl.min;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return min of values
- */
-public class MinLongAggregator extends AbstractMinAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8077547753784906280L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      internalAgg(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Merge the value, it will update the min aggregate value if aggregator
-   * passed as an argument will have value less than aggVal
-   *
-   * @param aggregator MinAggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    MinAggregator minAggregator = (MinAggregator) aggregator;
-    if (!aggregator.isFirstTime()) {
-      agg(minAggregator.aggVal);
-      firstTime = false;
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    MinAggregator aggregator = new MinAggregator();
-    aggregator.aggVal = aggVal;
-    aggregator.firstTime = firstTime;
-    return aggregator;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new MinLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java
deleted file mode 100644
index b901878..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java
+++ /dev/null
@@ -1,179 +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.carbondata.query.aggregator.impl.sum;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-public class SumBigDecimalAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 623750056131364540L;
-
-  /**
-   * aggregate value
-   */
-  private BigDecimal aggVal;
-
-  public SumBigDecimalAggregator() {
-    aggVal = new BigDecimal(0);
-    firstTime = false;
-  }
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (firstTime) {
-      aggVal = (BigDecimal) newVal;
-      firstTime = false;
-    } else {
-      aggVal = aggVal.add((BigDecimal) newVal);
-    }
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      BigDecimal value = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-      aggVal = aggVal.add(value);
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    byte[] bytes = DataTypeUtil.bigDecimalToByte(aggVal);
-    ByteBuffer allocate = ByteBuffer.allocate(4 + bytes.length);
-
-    allocate.putInt(bytes.length);
-    allocate.put(bytes);
-    allocate.rewind();
-    return allocate.array();
-  }
-
-  /**
-   * This method will return aggVal
-   *
-   * @return sum value
-   */
-  @Override public BigDecimal getBigDecimalValue() {
-    return aggVal;
-  }
-
-  /* Merge the value, it will update the sum aggregate value it will add new
-   * value to aggVal
-   *
-   * @param aggregator
-   *            SumAggregator
-   *
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    if (!aggregator.isFirstTime()) {
-      agg(aggregator.getBigDecimalValue());
-    }
-  }
-
-  /**
-   * This method return the sum value as an object
-   *
-   * @return sum value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (BigDecimal) newValue;
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = new BigDecimal(inPut.readUTF());
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeUTF(aggVal.toString());
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumBigDecimalAggregator aggr = new SumBigDecimalAggregator();
-    aggr.aggVal = aggVal;
-    aggr.firstTime = firstTime;
-    return aggr;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    byte[] valueByte = new byte[buffer.getInt()];
-    buffer.get(valueByte);
-    BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-    aggVal = aggVal.add(valueBigDecimal);
-    firstTime = false;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    BigDecimal value = getBigDecimalValue();
-    BigDecimal otherVal = o.getBigDecimalValue();
-    return value.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof SumBigDecimalAggregator)) {
-      return false;
-    }
-    SumBigDecimalAggregator o = (SumBigDecimalAggregator) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new SumBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java
deleted file mode 100644
index 777318d..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java
+++ /dev/null
@@ -1,179 +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.carbondata.query.aggregator.impl.sum;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-public class SumDoubleAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 623750056131364540L;
-
-  /**
-   * aggregate value
-   */
-  private double aggVal;
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(double newVal) {
-    aggVal += newVal;
-    firstTime = false;
-  }
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    aggVal += ((Number) newVal).doubleValue();
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal+= dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putDouble(aggVal);
-    return buffer.array();
-  }
-
-  /**
-   * This method will return aggVal
-   *
-   * @return sum value
-   */
-
-  @Override public Double getDoubleValue() {
-    return aggVal;
-  }
-
-  /* Merge the value, it will update the sum aggregate value it will add new
-   * value to aggVal
-   *
-   * @param aggregator  SumAggregator
-   *
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    if (!aggregator.isFirstTime()) {
-      agg(aggregator.getDoubleValue());
-    }
-  }
-
-  /**
-   * This method return the sum value as an object
-   *
-   * @return sum value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Double) newValue;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readDouble();
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeDouble(aggVal);
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumDoubleAggregator aggr = new SumDoubleAggregator();
-    aggr.aggVal = aggVal;
-    aggr.firstTime = firstTime;
-    return aggr;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    aggVal += ByteBuffer.wrap(value).getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    double value = getDoubleValue();
-    double otherVal = o.getDoubleValue();
-    if (value > otherVal) {
-      return 1;
-    }
-    if (value < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof SumDoubleAggregator)) {
-      return false;
-    }
-    SumDoubleAggregator o = (SumDoubleAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new SumDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java
deleted file mode 100644
index 7c245d9..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java
+++ /dev/null
@@ -1,165 +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.carbondata.query.aggregator.impl.sum;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-public class SumLongAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 623750056131364540L;
-
-  /**
-   * aggregate value
-   */
-  private long aggVal;
-
-  /**
-   * This method will update the aggVal it will add new value to aggVal
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    aggVal += (long) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal+= dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.LONG_SIZE_IN_BYTE);
-    buffer.putLong(aggVal);
-    return buffer.array();
-  }
-
-  /**
-   * This method will return aggVal
-   *
-   * @return sum value
-   */
-  @Override public Long getLongValue() {
-    return aggVal;
-  }
-
-  /* Merge the value, it will update the sum aggregate value it will add new
-   * value to aggVal
-   *
-   * @param aggregator SumAggregator
-   *
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    if (!aggregator.isFirstTime()) {
-      agg(aggregator.getLongValue());
-    }
-  }
-
-  /**
-   * This method return the sum value as an object
-   *
-   * @return sum long value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (long) newValue;
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readLong();
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeLong(aggVal);
-
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    SumLongAggregator aggr = new SumLongAggregator();
-    aggr.aggVal = aggVal;
-    aggr.firstTime = firstTime;
-    return aggr;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    aggVal += ByteBuffer.wrap(value).getLong();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    Long value = getLongValue();
-    Long otherVal = o.getLongValue();
-    if (value > otherVal) {
-      return 1;
-    }
-    if (value < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof SumLongAggregator)) {
-      return false;
-    }
-    SumLongAggregator o = (SumLongAggregator) obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new SumLongAggregator();
-  }
-}


[52/56] [abbrv] incubator-carbondata git commit: Fixed issue of dataframe query filters are not pushed down

Posted by jb...@apache.org.
Fixed issue of dataframe query filters are not pushed down


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

Branch: refs/heads/master
Commit: 8397673a81e5b9a54b6b5599208d53239732ee95
Parents: 3c41deb
Author: ravipesala <ra...@gmail.com>
Authored: Tue Jun 21 14:47:36 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Jun 21 14:47:36 2016 +0530

----------------------------------------------------------------------
 .../scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/8397673a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
index fc74291..6e3e8f8 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala
@@ -449,7 +449,7 @@ class CarbonOptimizer(optimizer: Optimizer, conf: CatalystConf)
   // get the carbon relation from plan.
   def collectCarbonRelation(plan: LogicalPlan): Seq[CarbonDecoderRelation] = {
     plan collect {
-      case Subquery(alias, l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)) =>
+      case l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _) =>
         CarbonDecoderRelation(l.attributeMap, carbonRelation)
     }
   }


[05/56] [abbrv] incubator-carbondata git commit: [Issue 618]Supported Spark 1.6 in Carbondata (#670)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java
new file mode 100644
index 0000000..0629007
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java
@@ -0,0 +1,103 @@
+/*
+ * 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.carbondata.query.aggregator.impl.distinct;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public abstract class AbstractDistinctCountAggregatorObjectSet implements MeasureAggregator {
+
+  private static final long serialVersionUID = 6313463368629960186L;
+
+  protected Set<Object> valueSetForObj;
+
+  public AbstractDistinctCountAggregatorObjectSet() {
+    valueSetForObj = new HashSet<Object>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * just need to add the unique values to agg set
+   */
+  @Override public void agg(double newVal) {
+  }
+
+  /**
+   * Distinct count Aggregate function which update the Distinct count
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    valueSetForObj.add(newVal);
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    return null;
+  }
+
+  @Override public Double getDoubleValue() {
+    return (double) valueSetForObj.size();
+  }
+
+  @Override public Long getLongValue() {
+    return (long) valueSetForObj.size();
+  }
+
+  @Override public BigDecimal getBigDecimalValue() {
+    return new BigDecimal(valueSetForObj.size());
+  }
+
+  @Override public Object getValueObject() {
+    return valueSetForObj.size();
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    valueSetForObj.add(newValue);
+  }
+
+  @Override public boolean isFirstTime() {
+    return false;
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+
+  }
+
+  public String toString() {
+    return valueSetForObj.size() + "";
+  }
+
+  @Override public void merge(byte[] value) {
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java
new file mode 100644
index 0000000..1b2b33d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java
@@ -0,0 +1,319 @@
+/*
+ * 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.carbondata.query.aggregator.impl.distinct;
+
+import java.io.ByteArrayInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+import org.roaringbitmap.IntIterator;
+import org.roaringbitmap.RoaringBitmap;
+
+/**
+ * * The distinct count aggregator
+ * Ex:
+ * ID NAME Sales
+ * <p>1 a 200
+ * <p>2 a 100
+ * <p>3 a 200
+ * select count(distinct sales) # would result 2
+ * select count(sales) # would result 3
+ */
+public class DistinctCountAggregator implements MeasureAggregator {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DistinctCountAggregator.class.getName());
+  /**
+   *
+   */
+  private static final long serialVersionUID = 6313463368629960186L;
+  /**
+   * For Spark CARBON to avoid heavy object transfer it better to flatten
+   * the Aggregators. There is no aggregation expected after setting this value.
+   */
+  private Double computedFixedValue;
+  /**
+   *
+   */
+  //    private Set<Double> valueSet;
+  private RoaringBitmap valueSet;
+
+  private byte[] data;
+
+  private double minValue;
+
+  public DistinctCountAggregator(Object minValue) {
+    valueSet = new RoaringBitmap();
+    if (minValue instanceof BigDecimal) {
+      this.minValue = ((BigDecimal) minValue).doubleValue();
+    } else if (minValue instanceof Long) {
+      this.minValue = ((Long) minValue).doubleValue();
+    } else {
+      this.minValue = (Double) minValue;
+    }
+  }
+
+  public DistinctCountAggregator() {
+    valueSet = new RoaringBitmap();
+  }
+
+  /**
+   * just need to add the unique values to agg set
+   */
+  @Override public void agg(double newVal) {
+    valueSet.add((int) (newVal - minValue));
+  }
+
+  /**
+   * Distinct count Aggregate function which update the Distinct count
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    if (newVal instanceof byte[]) {
+      byte[] values = (byte[]) newVal;
+      ByteBuffer buffer = ByteBuffer.wrap(values);
+      buffer.rewind();
+      while (buffer.hasRemaining()) {
+        valueSet.add(buffer.getInt());
+      }
+      return;
+    } else {
+      double value = new Double(newVal.toString());
+      agg(value);
+    }
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      valueSet.add((int) dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    if (valueSet.getCardinality() == 0) {
+      return new byte[0];
+    }
+    IntIterator iterator = valueSet.getIntIterator();
+    ByteBuffer buffer = ByteBuffer.allocate(valueSet.getCardinality() * 4 + 8);
+    buffer.putDouble(minValue);
+    while (iterator.hasNext()) {
+      buffer.putInt(iterator.next());
+    }
+    buffer.rewind();
+    return buffer.array();
+  }
+
+  private void agg(RoaringBitmap set2, double minValue) {
+    if (this.minValue == minValue) {
+      valueSet.or(set2);
+    } else {
+      if (this.minValue > minValue) {
+        IntIterator intIterator = valueSet.getIntIterator();
+        while (intIterator.hasNext()) {
+          set2.add((int) ((double) (intIterator.next() + this.minValue) - minValue));
+        }
+        this.minValue = minValue;
+        this.valueSet = set2;
+      } else {
+        IntIterator intIterator = set2.getIntIterator();
+        while (intIterator.hasNext()) {
+          valueSet.add((int) ((double) (intIterator.next() + minValue) - this.minValue));
+        }
+      }
+    }
+  }
+
+  /**
+   * merge the valueset so that we get the count of unique values
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    DistinctCountAggregator distinctCountAggregator = (DistinctCountAggregator) aggregator;
+    readData();
+    distinctCountAggregator.readData();
+    if (distinctCountAggregator.valueSet != null) {
+      agg(distinctCountAggregator.valueSet, distinctCountAggregator.minValue);
+    }
+  }
+
+  @Override public Double getDoubleValue() {
+    if (computedFixedValue == null) {
+      readData();
+      return (double) valueSet.getCardinality();
+    }
+    return computedFixedValue;
+  }
+
+  @Override public Long getLongValue() {
+    if (computedFixedValue == null) {
+      readData();
+      return (long) valueSet.getCardinality();
+    }
+    return computedFixedValue.longValue();
+  }
+
+  @Override public BigDecimal getBigDecimalValue() {
+    if (computedFixedValue == null) {
+      readData();
+      return new BigDecimal(valueSet.getCardinality());
+    }
+    return new BigDecimal(computedFixedValue);
+  }
+
+  @Override public Object getValueObject() {
+    return valueSet.getCardinality();
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    computedFixedValue = (Double) newValue;
+    valueSet = null;
+  }
+
+  @Override public boolean isFirstTime() {
+    return false;
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+
+    if (computedFixedValue != null) {
+      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
+      byteBuffer.putInt(-1);
+      byteBuffer.putDouble(computedFixedValue);
+      byteBuffer.flip();
+      output.write(byteBuffer.array());
+    } else {
+      if (valueSet != null) {
+        valueSet.serialize(output);
+      } else {
+        output.write(data);
+      }
+    }
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    valueSet = new RoaringBitmap();
+    valueSet.deserialize(inPut);
+  }
+
+  private void readData() {
+    if (data != null && (valueSet == null || valueSet.isEmpty())) {
+      ByteArrayInputStream stream = new ByteArrayInputStream(data);
+      DataInputStream outputStream = new DataInputStream(stream);
+      try {
+        readData(outputStream);
+        outputStream.close();
+        data = null;
+      } catch (IOException e) {
+        LOGGER.error(e, e.getMessage());
+      }
+    }
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    DistinctCountAggregator aggr = new DistinctCountAggregator(minValue);
+    aggr.valueSet = valueSet.clone();
+    return aggr;
+  }
+
+  @Override public int compareTo(MeasureAggregator measureAggr) {
+    double compFixedVal = getDoubleValue();
+    double otherVal = measureAggr.getDoubleValue();
+    if (compFixedVal > otherVal) {
+      return 1;
+    }
+    if (compFixedVal < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof DistinctCountAggregator)) {
+      return false;
+    }
+    DistinctCountAggregator o = (DistinctCountAggregator) obj;
+    return getDoubleValue().equals(o.getDoubleValue());
+  }
+
+  @Override public int hashCode() {
+    return getDoubleValue().hashCode();
+  }
+
+  @Override public MeasureAggregator get() {
+    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+    DataOutputStream outputStream = new DataOutputStream(byteStream);
+    try {
+      writeData(outputStream);
+    } catch (IOException ex) {
+      LOGGER.error(ex, ex.getMessage());
+    }
+    data = byteStream.toByteArray();
+    valueSet = null;
+    return this;
+  }
+
+  public String toString() {
+    if (computedFixedValue == null) {
+      readData();
+      return valueSet.getCardinality() + "";
+    }
+    return computedFixedValue + "";
+  }
+
+  public RoaringBitmap getBitMap() {
+    return valueSet;
+  }
+
+  public double getMinValue() {
+    return minValue;
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    buffer.rewind();
+    double currentMinValue = buffer.getDouble();
+    while (buffer.hasRemaining()) {
+      agg(buffer.getInt() + currentMinValue);
+    }
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new DistinctCountAggregator();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.java
new file mode 100644
index 0000000..3b26e53
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.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.carbondata.query.aggregator.impl.distinct;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class DistinctCountAggregatorObjectSet extends AbstractDistinctCountAggregatorObjectSet {
+
+  private static final long serialVersionUID = 6313463368629960186L;
+
+  /**
+   * just need to add the unique values to agg set
+   */
+  @Override public void agg(double newVal) {
+    valueSetForObj.add(newVal);
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
+    }
+  }
+
+  private void agg(Set<Object> set2) {
+    valueSetForObj.addAll(set2);
+  }
+
+  /**
+   * merge the valueset so that we get the count of unique values
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    DistinctCountAggregatorObjectSet distinctCountAggregator =
+        (DistinctCountAggregatorObjectSet) aggregator;
+    agg(distinctCountAggregator.valueSetForObj);
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    DistinctCountAggregatorObjectSet aggregator = new DistinctCountAggregatorObjectSet();
+    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
+    return aggregator;
+  }
+
+  @Override public int compareTo(MeasureAggregator measureAggr) {
+    double valueSetForObjSize = getDoubleValue();
+    double otherVal = measureAggr.getDoubleValue();
+    if (valueSetForObjSize > otherVal) {
+      return 1;
+    }
+    if (valueSetForObjSize < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof DistinctCountAggregatorObjectSet)) {
+      return false;
+    }
+    DistinctCountAggregatorObjectSet o = (DistinctCountAggregatorObjectSet) obj;
+    return getDoubleValue().equals(o.getDoubleValue());
+  }
+
+  @Override public int hashCode() {
+    return getDoubleValue().hashCode();
+  }
+
+  @Override public MeasureAggregator get() {
+    return this;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new DistinctCountAggregatorObjectSet();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.java
new file mode 100644
index 0000000..2f44bca
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountBigDecimalAggregatorObjectSet.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.carbondata.query.aggregator.impl.distinct;
+
+import java.math.BigDecimal;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class DistinctCountBigDecimalAggregatorObjectSet
+    extends AbstractDistinctCountAggregatorObjectSet {
+
+  private static final long serialVersionUID = 6313463368629960186L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
+    }
+  }
+
+  private void agg(Set<Object> set2) {
+    valueSetForObj.addAll(set2);
+  }
+
+  /**
+   * merge the valueset so that we get the count of unique values
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    DistinctCountBigDecimalAggregatorObjectSet distinctCountBigDecimalAggregatorObjectSet =
+        (DistinctCountBigDecimalAggregatorObjectSet) aggregator;
+    agg(distinctCountBigDecimalAggregatorObjectSet.valueSetForObj);
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    DistinctCountBigDecimalAggregatorObjectSet aggregator =
+        new DistinctCountBigDecimalAggregatorObjectSet();
+    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
+    return aggregator;
+  }
+
+  @Override public int compareTo(MeasureAggregator measureAggr) {
+    BigDecimal valueSetForObjSize = getBigDecimalValue();
+    BigDecimal otherVal = measureAggr.getBigDecimalValue();
+    return valueSetForObjSize.compareTo(otherVal);
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof DistinctCountBigDecimalAggregatorObjectSet)) {
+      return false;
+    }
+    DistinctCountBigDecimalAggregatorObjectSet o = (DistinctCountBigDecimalAggregatorObjectSet) obj;
+    return getBigDecimalValue().equals(o.getBigDecimalValue());
+  }
+
+  @Override public int hashCode() {
+    return getBigDecimalValue().hashCode();
+  }
+
+  @Override public MeasureAggregator get() {
+    return this;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new DistinctCountBigDecimalAggregatorObjectSet();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java
new file mode 100644
index 0000000..c4f7216
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountLongAggregatorObjectSet.java
@@ -0,0 +1,89 @@
+/*
+ * 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.carbondata.query.aggregator.impl.distinct;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class DistinctCountLongAggregatorObjectSet extends AbstractDistinctCountAggregatorObjectSet {
+
+  private static final long serialVersionUID = 6313463368629960186L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
+    }
+  }
+
+  private void agg(Set<Object> set2) {
+    valueSetForObj.addAll(set2);
+  }
+
+  /**
+   * merge the valueset so that we get the count of unique values
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    DistinctCountLongAggregatorObjectSet distinctCountAggregator =
+        (DistinctCountLongAggregatorObjectSet) aggregator;
+    agg(distinctCountAggregator.valueSetForObj);
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    DistinctCountLongAggregatorObjectSet aggregator = new DistinctCountLongAggregatorObjectSet();
+    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
+    return aggregator;
+  }
+
+  @Override public int compareTo(MeasureAggregator measureAggr) {
+    long valueSetForObjSize = getLongValue();
+    long otherVal = measureAggr.getLongValue();
+    if (valueSetForObjSize > otherVal) {
+      return 1;
+    }
+    if (valueSetForObjSize < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof DistinctCountLongAggregatorObjectSet)) {
+      return false;
+    }
+    DistinctCountLongAggregatorObjectSet o = (DistinctCountLongAggregatorObjectSet) obj;
+    return getLongValue().equals(o.getLongValue());
+  }
+
+  @Override public int hashCode() {
+    return getLongValue().hashCode();
+  }
+
+  @Override public MeasureAggregator get() {
+    return this;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new DistinctCountLongAggregatorObjectSet();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java
new file mode 100644
index 0000000..e3d4623
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctStringCountAggregator.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.distinct;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.HashSet;
+import java.util.Set;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class DistinctStringCountAggregator implements MeasureAggregator {
+  private static final long serialVersionUID = 6313463368629960186L;
+
+  private Set<String> valueSetForStr;
+
+  public DistinctStringCountAggregator() {
+    this.valueSetForStr = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  public void agg(double newVal) {
+  }
+
+  public void agg(String newVal) {
+    this.valueSetForStr.add(newVal);
+  }
+
+  private void agg(Set<String> set2) {
+    this.valueSetForStr.addAll(set2);
+  }
+
+  public void merge(MeasureAggregator aggregator) {
+    DistinctStringCountAggregator distinctCountAggregator =
+        (DistinctStringCountAggregator) aggregator;
+    agg(distinctCountAggregator.valueSetForStr);
+  }
+
+  public Double getDoubleValue() {
+    return (double) this.valueSetForStr.size();
+  }
+
+  public Long getLongValue() {
+    return (long) this.valueSetForStr.size();
+  }
+
+  public BigDecimal getBigDecimalValue() {
+    return new BigDecimal(this.valueSetForStr.size());
+  }
+
+  public Object getValueObject() {
+    return Integer.valueOf(this.valueSetForStr.size());
+  }
+
+  public void setNewValue(Object newValue) {
+  }
+
+  public boolean isFirstTime() {
+    return false;
+  }
+
+  public void writeData(DataOutput output) throws IOException {
+    int length = this.valueSetForStr.size() * 8;
+    ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4);
+    byteBuffer.putInt(length);
+    for (String val : this.valueSetForStr) {
+      byte[] b = val.getBytes(Charset.defaultCharset());
+      byteBuffer.putInt(b.length);
+      byteBuffer.put(b);
+    }
+    byteBuffer.flip();
+    output.write(byteBuffer.array());
+  }
+
+  public void readData(DataInput inPut) throws IOException {
+    int length = inPut.readInt();
+    length /= 8;
+    this.valueSetForStr = new HashSet<String>(length + 1, 1.0F);
+    for (int i = 0; i < length; i++) {
+      byte[] b = new byte[inPut.readInt()];
+      inPut.readFully(b);
+      this.valueSetForStr.add(new String(b, Charset.defaultCharset()));
+    }
+  }
+
+  public MeasureAggregator getCopy() {
+    DistinctStringCountAggregator aggregator = new DistinctStringCountAggregator();
+    aggregator.valueSetForStr = new HashSet<String>(this.valueSetForStr);
+    return aggregator;
+  }
+
+  public int compareTo(MeasureAggregator o) {
+    double val = getDoubleValue();
+    double otherVal = o.getDoubleValue();
+    if (val > otherVal) {
+      return 1;
+    }
+    if (val < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof DistinctStringCountAggregator)) {
+      return false;
+    }
+    DistinctStringCountAggregator o = (DistinctStringCountAggregator) obj;
+    return getDoubleValue().equals(o.getDoubleValue());
+  }
+
+  @Override public int hashCode() {
+    return getDoubleValue().hashCode();
+  }
+
+  @Override public void agg(Object newVal) {
+    this.valueSetForStr.add((String) newVal);
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+  }
+
+  @Override public byte[] getByteArray() {
+    return null;
+  }
+
+  @Override public MeasureAggregator get() {
+    return this;
+  }
+
+  public String toString() {
+    return valueSetForStr.size() + "";
+  }
+
+  @Override public void merge(byte[] value) {
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new DistinctStringCountAggregator();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java
new file mode 100644
index 0000000..6a59ec9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctBigDecimalAggregator.java
@@ -0,0 +1,233 @@
+/*
+ * 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.carbondata.query.aggregator.impl.distinct;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+import java.nio.charset.Charset;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.DataTypeUtil;
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+/**
+ * The sum distinct aggregator
+ * Ex:
+ * ID NAME Sales
+ * 1 a 200
+ * 2 a 100
+ * 3 a 200
+ * select sum(distinct sales) # would result 300
+ */
+public class SumDistinctBigDecimalAggregator extends AbstractMeasureAggregatorBasic {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 6313463368629960155L;
+
+  /**
+   * For Spark CARBON to avoid heavy object transfer it better to flatten the
+   * Aggregators. There is no aggregation expected after setting this value.
+   */
+  private BigDecimal computedFixedValue;
+
+  /**
+   *
+   */
+  private Set<BigDecimal> valueSet;
+
+  public SumDistinctBigDecimalAggregator() {
+    valueSet = new HashSet<BigDecimal>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * Distinct Aggregate function which update the Distinct set
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    valueSet.add(
+        newVal instanceof BigDecimal ? (BigDecimal) newVal : new BigDecimal(newVal.toString()));
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      valueSet.add(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    Iterator<BigDecimal> iterator = valueSet.iterator();
+    ByteBuffer buffer =
+        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    while (iterator.hasNext()) {
+      byte[] bytes = DataTypeUtil.bigDecimalToByte(iterator.next());
+      buffer.putInt(bytes.length);
+      buffer.put(bytes);
+    }
+    buffer.rewind();
+    return buffer.array();
+  }
+
+  private void agg(Set<BigDecimal> set2) {
+    valueSet.addAll(set2);
+  }
+
+  /**
+   * merge the valueset so that we get the count of unique values
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    SumDistinctBigDecimalAggregator distinctAggregator =
+        (SumDistinctBigDecimalAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(distinctAggregator.valueSet);
+      firstTime = false;
+    }
+  }
+
+  @Override public BigDecimal getBigDecimalValue() {
+    if (computedFixedValue == null) {
+      BigDecimal result = new BigDecimal(0);
+      for (BigDecimal aValue : valueSet) {
+        result = result.add(aValue);
+      }
+      return result;
+    }
+    return computedFixedValue;
+  }
+
+  @Override public Object getValueObject() {
+    return getBigDecimalValue();
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    computedFixedValue = (BigDecimal) newValue;
+    valueSet = null;
+  }
+
+  @Override public boolean isFirstTime() {
+    return firstTime;
+  }
+
+  @Override public void writeData(DataOutput dataOutput) throws IOException {
+    if (computedFixedValue != null) {
+      byte[] bytes = DataTypeUtil.bigDecimalToByte(computedFixedValue);
+      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 4 + bytes.length);
+      byteBuffer.putInt(-1);
+      byteBuffer.putInt(bytes.length);
+      byteBuffer.put(bytes);
+      byteBuffer.flip();
+      dataOutput.write(byteBuffer.array());
+    } else {
+      int length = valueSet.size() * 8 + valueSet.size() * 4;
+      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
+      byteBuffer.putInt(length);
+      for (BigDecimal val : valueSet) {
+        byte[] bytes =
+            val.toString().getBytes(Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET));
+        byteBuffer.putInt(-1);
+        byteBuffer.putInt(bytes.length);
+        byteBuffer.put(bytes);
+      }
+      byteBuffer.flip();
+      dataOutput.write(byteBuffer.array());
+    }
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    int length = inPut.readInt();
+
+    if (length == -1) {
+      computedFixedValue = new BigDecimal(inPut.readUTF());
+      valueSet = null;
+    } else {
+      length = length / 8;
+      valueSet = new HashSet<BigDecimal>(length + 1, 1.0f);
+      for (int i = 0; i < length; i++) {
+        valueSet.add(new BigDecimal(inPut.readUTF()));
+      }
+    }
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    buffer.rewind();
+    while (buffer.hasRemaining()) {
+      byte[] valueByte = new byte[buffer.getInt()];
+      buffer.get(valueByte);
+      BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
+      agg(valueBigDecimal);
+    }
+  }
+
+  public String toString() {
+    if (computedFixedValue == null) {
+      return valueSet.size() + "";
+    }
+    return computedFixedValue + "";
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    SumDistinctBigDecimalAggregator aggregator = new SumDistinctBigDecimalAggregator();
+    aggregator.valueSet = new HashSet<BigDecimal>(valueSet);
+    return aggregator;
+  }
+
+  @Override public int compareTo(MeasureAggregator msr) {
+    BigDecimal msrValObj = getBigDecimalValue();
+    BigDecimal otherVal = msr.getBigDecimalValue();
+
+    return msrValObj.compareTo(otherVal);
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof SumDistinctBigDecimalAggregator)) {
+      return false;
+    }
+    SumDistinctBigDecimalAggregator o = (SumDistinctBigDecimalAggregator) obj;
+    return getBigDecimalValue().equals(o.getBigDecimalValue());
+  }
+
+  @Override public int hashCode() {
+    return getBigDecimalValue().hashCode();
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new SumDistinctBigDecimalAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java
new file mode 100644
index 0000000..0229f24
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctDoubleAggregator.java
@@ -0,0 +1,231 @@
+/*
+ * 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.carbondata.query.aggregator.impl.distinct;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+/**
+ * The sum distinct aggregator
+ * Ex:
+ * ID NAME Sales
+ * 1 a 200
+ * 2 a 100
+ * 3 a 200
+ * select sum(distinct sales) # would result 300
+ */
+
+public class SumDistinctDoubleAggregator extends AbstractMeasureAggregatorBasic {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 6313463368629960155L;
+
+  /**
+   * For Spark CARBON to avoid heavy object transfer it better to flatten the
+   * Aggregators. There is no aggregation expected after setting this value.
+   */
+  private Double computedFixedValue;
+
+  /**
+   *
+   */
+  private Set<Double> valueSet;
+
+  public SumDistinctDoubleAggregator() {
+    valueSet = new HashSet<Double>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * just need to add the unique values to agg set
+   */
+  @Override public void agg(double newVal) {
+    valueSet.add(newVal);
+    firstTime = false;
+  }
+
+  /**
+   * Distinct Aggregate function which update the Distinct set
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    valueSet.add(newVal instanceof Double ? (Double) newVal : new Double(newVal.toString()));
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      valueSet.add(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    Iterator<Double> iterator = valueSet.iterator();
+    ByteBuffer buffer =
+        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    while (iterator.hasNext()) {
+      buffer.putDouble(iterator.next());
+    }
+    buffer.rewind();
+    return buffer.array();
+  }
+
+  private void agg(Set<Double> set2) {
+    valueSet.addAll(set2);
+  }
+
+  /**
+   * merge the valueset so that we get the count of unique values
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    SumDistinctDoubleAggregator distinctAggregator = (SumDistinctDoubleAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(distinctAggregator.valueSet);
+      firstTime = false;
+    }
+  }
+
+  @Override public Double getDoubleValue() {
+    if (computedFixedValue == null) {
+      double result = 0;
+      for (Double aValue : valueSet) {
+        result += aValue;
+      }
+      return result;
+    }
+    return computedFixedValue;
+  }
+
+  @Override public Object getValueObject() {
+    return getDoubleValue();
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    computedFixedValue = (Double) newValue;
+    valueSet = null;
+  }
+
+  @Override public boolean isFirstTime() {
+    return firstTime;
+  }
+
+  @Override public void writeData(DataOutput dataOutput) throws IOException {
+    if (computedFixedValue != null) {
+      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
+      byteBuffer.putInt(-1);
+      byteBuffer.putDouble(computedFixedValue);
+      byteBuffer.flip();
+      dataOutput.write(byteBuffer.array());
+    } else {
+      int length = valueSet.size() * 8;
+      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
+      byteBuffer.putInt(length);
+      for (double val : valueSet) {
+        byteBuffer.putDouble(val);
+      }
+      byteBuffer.flip();
+      dataOutput.write(byteBuffer.array());
+    }
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    int length = inPut.readInt();
+
+    if (length == -1) {
+      computedFixedValue = inPut.readDouble();
+      valueSet = null;
+    } else {
+      length = length / 8;
+      valueSet = new HashSet<Double>(length + 1, 1.0f);
+      for (int i = 0; i < length; i++) {
+        valueSet.add(inPut.readDouble());
+      }
+    }
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    buffer.rewind();
+    while (buffer.hasRemaining()) {
+      agg(buffer.getDouble());
+    }
+  }
+
+  public String toString() {
+    if (computedFixedValue == null) {
+      return valueSet.size() + "";
+    }
+    return computedFixedValue + "";
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    SumDistinctDoubleAggregator aggregator = new SumDistinctDoubleAggregator();
+    aggregator.valueSet = new HashSet<Double>(valueSet);
+    return aggregator;
+  }
+
+  @Override public int compareTo(MeasureAggregator msr) {
+    double msrValObj = getDoubleValue();
+    double otherVal = msr.getDoubleValue();
+    if (msrValObj > otherVal) {
+      return 1;
+    }
+    if (msrValObj < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof SumDistinctDoubleAggregator)) {
+      return false;
+    }
+    SumDistinctDoubleAggregator o = (SumDistinctDoubleAggregator) obj;
+    return getDoubleValue().equals(o.getDoubleValue());
+  }
+
+  @Override public int hashCode() {
+    return getDoubleValue().hashCode();
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new SumDistinctDoubleAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java
new file mode 100644
index 0000000..d57c34b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/SumDistinctLongAggregator.java
@@ -0,0 +1,221 @@
+/*
+ * 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.carbondata.query.aggregator.impl.distinct;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.Set;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+/**
+ * The sum distinct aggregator
+ * Ex:
+ * ID NAME Sales
+ * 1 a 200
+ * 2 a 100
+ * 3 a 200
+ * select sum(distinct sales) # would result 300
+ */
+
+public class SumDistinctLongAggregator extends AbstractMeasureAggregatorBasic {
+
+  private static final long serialVersionUID = 6313463368629960155L;
+
+  /**
+   * For Spark CARBON to avoid heavy object transfer it better to flatten the
+   * Aggregators. There is no aggregation expected after setting this value.
+   */
+  private Long computedFixedValue;
+
+  /**
+   *
+   */
+  private Set<Long> valueSet;
+
+  public SumDistinctLongAggregator() {
+    valueSet = new HashSet<Long>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * Distinct Aggregate function which update the Distinct set
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    valueSet.add(newVal instanceof Long ? (Long) newVal : Long.valueOf(newVal.toString()));
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      valueSet.add(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    Iterator<Long> iterator = valueSet.iterator();
+    ByteBuffer buffer =
+        ByteBuffer.allocate(valueSet.size() * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    while (iterator.hasNext()) {
+      buffer.putLong(iterator.next());
+    }
+    buffer.rewind();
+    return buffer.array();
+  }
+
+  private void agg(Set<Long> set2) {
+    valueSet.addAll(set2);
+  }
+
+  /**
+   * merge the valueset so that we get the count of unique values
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    SumDistinctLongAggregator distinctAggregator = (SumDistinctLongAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(distinctAggregator.valueSet);
+      firstTime = false;
+    }
+  }
+
+  @Override public Long getLongValue() {
+    if (computedFixedValue == null) {
+      long result = 0;
+      for (Long aValue : valueSet) {
+        result += aValue;
+      }
+      return result;
+    }
+    return computedFixedValue;
+  }
+
+  @Override public Object getValueObject() {
+    return getLongValue();
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    computedFixedValue = (Long) newValue;
+    valueSet = null;
+  }
+
+  @Override public boolean isFirstTime() {
+    return firstTime;
+  }
+
+  @Override public void writeData(DataOutput dataOutput) throws IOException {
+    if (computedFixedValue != null) {
+      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
+      byteBuffer.putInt(-1);
+      byteBuffer.putLong(computedFixedValue);
+      byteBuffer.flip();
+      dataOutput.write(byteBuffer.array());
+    } else {
+      int length = valueSet.size() * 8;
+      ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4 + 1);
+      byteBuffer.putInt(length);
+      for (long val : valueSet) {
+        byteBuffer.putLong(val);
+      }
+      byteBuffer.flip();
+      dataOutput.write(byteBuffer.array());
+    }
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    int length = inPut.readInt();
+
+    if (length == -1) {
+      computedFixedValue = inPut.readLong();
+      valueSet = null;
+    } else {
+      length = length / 8;
+      valueSet = new HashSet<Long>(length + 1, 1.0f);
+      for (int i = 0; i < length; i++) {
+        valueSet.add(inPut.readLong());
+      }
+    }
+
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    buffer.rewind();
+    while (buffer.hasRemaining()) {
+      agg(buffer.getLong());
+    }
+  }
+
+  public String toString() {
+    if (computedFixedValue == null) {
+      return valueSet.size() + "";
+    }
+    return computedFixedValue + "";
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    SumDistinctLongAggregator aggregator = new SumDistinctLongAggregator();
+    aggregator.valueSet = new HashSet<Long>(valueSet);
+    return aggregator;
+  }
+
+  @Override public int compareTo(MeasureAggregator msr) {
+    long msrValObj = getLongValue();
+    long otherVal = msr.getLongValue();
+    if (msrValObj > otherVal) {
+      return 1;
+    }
+    if (msrValObj < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof SumDistinctLongAggregator)) {
+      return false;
+    }
+    SumDistinctLongAggregator o = (SumDistinctLongAggregator) obj;
+    return getLongValue().equals(o.getLongValue());
+  }
+
+  @Override public int hashCode() {
+    return getLongValue().hashCode();
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new SumDistinctLongAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java
new file mode 100644
index 0000000..f423085
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/AbstractMeasureAggregatorDummy.java
@@ -0,0 +1,70 @@
+/*
+ * 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.carbondata.query.aggregator.impl.dummy;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+/**
+ * AbstractMeasureAggregatorDummy
+ * Used for custom Carbon Aggregator dummy
+ */
+public abstract class AbstractMeasureAggregatorDummy extends AbstractMeasureAggregatorBasic {
+  private static final long serialVersionUID = 1L;
+
+  @Override public int compareTo(MeasureAggregator o) {
+    if (equals(o)) {
+      return 0;
+    }
+    return -1;
+  }
+
+  @Override public boolean equals(Object arg0) {
+    return super.equals(arg0);
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode();
+  }
+
+  @Override public byte[] getByteArray() {
+    return null;
+  }
+
+  @Override public void merge(MeasureAggregator aggregator) {
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    return null;
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+  }
+
+  @Override public void merge(byte[] value) {
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java
new file mode 100644
index 0000000..8a33fe1
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyBigDecimalAggregator.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.dummy;
+
+import java.math.BigDecimal;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class DummyBigDecimalAggregator extends AbstractMeasureAggregatorDummy {
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * aggregate value
+   */
+  private BigDecimal aggVal;
+
+  @Override public void agg(Object newVal) {
+    aggVal = (BigDecimal) newVal;
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+      firstTime = false;
+    }
+  }
+
+  @Override public BigDecimal getBigDecimalValue() {
+    return aggVal;
+  }
+
+  @Override public Object getValueObject() {
+    return aggVal;
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (BigDecimal) newValue;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new DummyBigDecimalAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java
new file mode 100644
index 0000000..cd4fe56
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyDoubleAggregator.java
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.query.aggregator.impl.dummy;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class DummyDoubleAggregator extends AbstractMeasureAggregatorDummy {
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * aggregate value
+   */
+  private double aggVal;
+
+  @Override public void agg(double newVal) {
+    aggVal = newVal;
+    firstTime = false;
+  }
+
+  @Override public void agg(Object newVal) {
+    aggVal = (Double) newVal;
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+      firstTime = false;
+    }
+  }
+
+  @Override public Double getDoubleValue() {
+    return aggVal;
+  }
+
+  @Override public Object getValueObject() {
+    return aggVal;
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (Double) newValue;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new DummyDoubleAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java
new file mode 100644
index 0000000..4131895
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/dummy/DummyLongAggregator.java
@@ -0,0 +1,59 @@
+/*
+ * 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.carbondata.query.aggregator.impl.dummy;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+public class DummyLongAggregator extends AbstractMeasureAggregatorDummy {
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * aggregate value
+   */
+  private long aggVal;
+
+  @Override public void agg(Object newVal) {
+    aggVal = (Long) newVal;
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+      firstTime = false;
+    }
+  }
+
+  @Override public Long getLongValue() {
+    return aggVal;
+  }
+
+  @Override public Object getValueObject() {
+    return aggVal;
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (Long) newValue;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new DummyLongAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java
new file mode 100644
index 0000000..27b1876
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/max/AbstractMaxAggregator.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.max;
+
+import java.io.ByteArrayInputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorMaxMin;
+
+public abstract class AbstractMaxAggregator extends AbstractMeasureAggregatorMaxMin {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractMaxAggregator.class.getName());
+
+  protected void internalAgg(Object value) {
+    if (value instanceof Comparable) {
+      @SuppressWarnings("unchecked") Comparable<Object> newValue = ((Comparable<Object>) value);
+      aggVal = (aggVal == null || aggVal.compareTo(newValue) < 0) ? newValue : aggVal;
+    }
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteArrayInputStream bytesInputStream = null;
+    ObjectInput in = null;
+    try {
+      bytesInputStream = new ByteArrayInputStream(value);
+      in = new ObjectInputStream(bytesInputStream);
+      Object newVal = (Comparable<Object>) in.readObject();
+      internalAgg(newVal);
+      firstTime = false;
+    } catch (Exception e) {
+      LOGGER.error(e, "Problem while merging byte array in maxAggregator: " + e.getMessage());
+    } finally {
+      CarbonUtil.closeStreams(bytesInputStream);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.java
new file mode 100644
index 0000000..d0d8af0
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxAggregator.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.carbondata.query.aggregator.impl.max;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description :
+ * It will return max of values
+ */
+public class MaxAggregator extends AbstractMaxAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -5850218739083899419L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      internalAgg(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Merge the value, it will update the max aggregate value if aggregator
+   * passed as an argument will have value greater than aggVal
+   *
+   * @param aggregator MaxAggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(maxAggregator.aggVal);
+      firstTime = false;
+    }
+
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    MaxAggregator aggregator = new MaxAggregator();
+    aggregator.aggVal = aggVal;
+    aggregator.firstTime = firstTime;
+    return aggregator;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new MaxAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java
new file mode 100644
index 0000000..c4149c6
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxBigDecimalAggregator.java
@@ -0,0 +1,67 @@
+/*
+ * 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.carbondata.query.aggregator.impl.max;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description :
+ * It will return max of values
+ */
+public class MaxBigDecimalAggregator extends AbstractMaxAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -5850218739083899419L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      internalAgg(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Merge the value, it will update the max aggregate value if aggregator
+   * passed as an argument will have value greater than aggVal
+   *
+   * @param aggregator MaxAggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(maxAggregator.aggVal);
+      firstTime = false;
+    }
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    MaxAggregator aggregator = new MaxAggregator();
+    aggregator.aggVal = aggVal;
+    aggregator.firstTime = firstTime;
+    return aggregator;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new MaxBigDecimalAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java
new file mode 100644
index 0000000..55b3be5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/max/MaxLongAggregator.java
@@ -0,0 +1,67 @@
+/*
+ * 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.carbondata.query.aggregator.impl.max;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description :
+ * It will return max of values
+ */
+public class MaxLongAggregator extends AbstractMaxAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -5850218739083899419L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      internalAgg(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Merge the value, it will update the max aggregate value if aggregator
+   * passed as an argument will have value greater than aggVal
+   *
+   * @param aggregator MaxAggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    MaxAggregator maxAggregator = (MaxAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(maxAggregator.aggVal);
+      firstTime = false;
+    }
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    MaxAggregator aggregator = new MaxAggregator();
+    aggregator.aggVal = aggVal;
+    aggregator.firstTime = firstTime;
+    return aggregator;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new MaxLongAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java
new file mode 100644
index 0000000..77aa163
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/min/AbstractMinAggregator.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.min;
+
+import java.io.ByteArrayInputStream;
+import java.io.ObjectInput;
+import java.io.ObjectInputStream;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorMaxMin;
+
+public abstract class AbstractMinAggregator extends AbstractMeasureAggregatorMaxMin {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractMinAggregator.class.getName());
+
+  protected void internalAgg(Object value) {
+    if (value instanceof Comparable) {
+      @SuppressWarnings("unchecked") Comparable<Object> newValue = ((Comparable<Object>) value);
+      aggVal = (aggVal == null || aggVal.compareTo(newValue) > 0) ? newValue : aggVal;
+    }
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    ByteArrayInputStream bis = null;
+    ObjectInput objectInput = null;
+    try {
+      bis = new ByteArrayInputStream(value);
+      objectInput = new ObjectInputStream(bis);
+      Object newVal = (Comparable<Object>) objectInput.readObject();
+      internalAgg(newVal);
+      firstTime = false;
+    } catch (Exception e) {
+      LOGGER.error(e, "Problem while merging byte array in minAggregator: " + e.getMessage());
+    } finally {
+      CarbonUtil.closeStreams(bis);
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java
new file mode 100644
index 0000000..4bb3d73
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinAggregator.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.min;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description : It will return min of values
+ */
+public class MinAggregator extends AbstractMinAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -8077547753784906280L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      internalAgg(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Merge the value, it will update the min aggregate value if aggregator
+   * passed as an argument will have value less than aggVal
+   *
+   * @param aggregator MinAggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    MinAggregator minAggregator = (MinAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(minAggregator.aggVal);
+      firstTime = false;
+    }
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    MinAggregator aggregator = new MinAggregator();
+    aggregator.aggVal = aggVal;
+    aggregator.firstTime = firstTime;
+    return aggregator;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new MinAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java
new file mode 100644
index 0000000..7347dc1
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinBigDecimalAggregator.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.min;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description : It will return min of values
+ */
+public class MinBigDecimalAggregator extends AbstractMinAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -8077547753784906280L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      internalAgg(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Merge the value, it will update the min aggregate value if aggregator
+   * passed as an argument will have value less than aggVal
+   *
+   * @param aggregator MinAggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    MinAggregator minAggregator = (MinAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(minAggregator.aggVal);
+      firstTime = false;
+    }
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    MinAggregator aggregator = new MinAggregator();
+    aggregator.aggVal = aggVal;
+    aggregator.firstTime = firstTime;
+    return aggregator;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new MinBigDecimalAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java
new file mode 100644
index 0000000..af614c9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/min/MinLongAggregator.java
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.min;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.query.aggregator.MeasureAggregator;
+
+/**
+ * Class Description : It will return min of values
+ */
+public class MinLongAggregator extends AbstractMinAggregator {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -8077547753784906280L;
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      internalAgg(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Merge the value, it will update the min aggregate value if aggregator
+   * passed as an argument will have value less than aggVal
+   *
+   * @param aggregator MinAggregator
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    MinAggregator minAggregator = (MinAggregator) aggregator;
+    if (!aggregator.isFirstTime()) {
+      agg(minAggregator.aggVal);
+      firstTime = false;
+    }
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    MinAggregator aggregator = new MinAggregator();
+    aggregator.aggVal = aggVal;
+    aggregator.firstTime = firstTime;
+    return aggregator;
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new MinLongAggregator();
+  }
+}


[30/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/BinaryConditionalExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/BinaryConditionalExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/BinaryConditionalExpression.java
new file mode 100644
index 0000000..0c74ebf
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/BinaryConditionalExpression.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.carbondata.scan.expression.conditional;
+
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.logical.BinaryLogicalExpression;
+
+public abstract class BinaryConditionalExpression extends BinaryLogicalExpression
+    implements ConditionalExpression {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+
+  public BinaryConditionalExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/ConditionalExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/ConditionalExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/ConditionalExpression.java
new file mode 100644
index 0000000..d9ed78f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/ConditionalExpression.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.conditional;
+
+import java.util.List;
+
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.ExpressionResult;
+
+public interface ConditionalExpression {
+
+  // Will get the column informations involved in the expressions by
+  // traversing the tree
+  List<ColumnExpression> getColumnList();
+
+  boolean isSingleDimension();
+
+  List<ExpressionResult> getLiterals();
+
+  /**
+   * will return the flag of direct dictionary column
+   *
+   * @return
+   */
+  boolean isDirectDictionaryColumns();
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
new file mode 100644
index 0000000..7b2a32d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
@@ -0,0 +1,95 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.conditional;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class EqualToExpression extends BinaryConditionalExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  public EqualToExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult elRes = left.evaluate(value);
+    ExpressionResult erRes = right.evaluate(value);
+
+    boolean result = false;
+
+    ExpressionResult val1 = elRes;
+    ExpressionResult val2 = erRes;
+
+    if (elRes.isNull() || erRes.isNull()) {
+      result = elRes.isNull() && erRes.isNull();
+      val1.set(DataType.BooleanType, result);
+      return val1;
+    }
+    //default implementation if the data types are different for the resultsets
+    if (elRes.getDataType() != erRes.getDataType()) {
+      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
+        val2 = elRes;
+        val1 = erRes;
+      }
+    }
+
+    // todo: move to util
+    switch (val1.getDataType()) {
+      case StringType:
+        result = val1.getString().equals(val2.getString());
+        break;
+      case IntegerType:
+        result = val1.getInt().equals(val2.getInt());
+        break;
+      case DoubleType:
+        result = val1.getDouble().equals(val2.getDouble());
+        break;
+      case TimestampType:
+        result = val1.getTime().equals(val2.getTime());
+        break;
+      case LongType:
+        result = val1.getLong().equals(val2.getLong());
+        break;
+      case DecimalType:
+        result = val1.getDecimal().compareTo(val2.getDecimal()) == 0;
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    }
+    val1.set(DataType.BooleanType, result);
+    return val1;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.EQUALS;
+  }
+
+  @Override public String getString() {
+    return "EqualTo(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
new file mode 100644
index 0000000..bb00467
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
@@ -0,0 +1,85 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.conditional;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class GreaterThanEqualToExpression extends BinaryConditionalExpression {
+  private static final long serialVersionUID = 4185317066280688984L;
+
+  public GreaterThanEqualToExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult elRes = left.evaluate(value);
+    ExpressionResult erRes = right.evaluate(value);
+    ExpressionResult exprResVal1 = elRes;
+    if (elRes.isNull() || erRes.isNull()) {
+      elRes.set(DataType.BooleanType, false);
+      return elRes;
+    }
+    if (elRes.getDataType() != erRes.getDataType()) {
+      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
+        exprResVal1 = erRes;
+      }
+
+    }
+    boolean result = false;
+    switch (exprResVal1.getDataType()) {
+      case StringType:
+        result = elRes.getString().compareTo(erRes.getString()) >= 0;
+        break;
+      case IntegerType:
+        result = elRes.getInt() >= (erRes.getInt());
+        break;
+      case DoubleType:
+        result = elRes.getDouble() >= (erRes.getDouble());
+        break;
+      case TimestampType:
+        result = elRes.getTime() >= (erRes.getTime());
+        break;
+      case LongType:
+        result = elRes.getLong() >= (erRes.getLong());
+        break;
+      case DecimalType:
+        result = elRes.getDecimal().compareTo(erRes.getDecimal()) >= 0;
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "DataType: " + exprResVal1.getDataType() + " not supported for the filter expression");
+    }
+    exprResVal1.set(DataType.BooleanType, result);
+    return exprResVal1;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.GREATERTHAN_EQUALTO;
+  }
+
+  @Override public String getString() {
+    return "GreaterThanEqualTo(" + left.getString() + ',' + right.getString() + ')';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
new file mode 100644
index 0000000..f1bde04
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
@@ -0,0 +1,89 @@
+/*
+ * 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.carbondata.scan.expression.conditional;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+
+public class GreaterThanExpression extends BinaryConditionalExpression {
+  private static final long serialVersionUID = -5319109756575539219L;
+
+  public GreaterThanExpression(Expression left, Expression right) {
+    super(left, right);
+    // TODO Auto-generated constructor stub
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult exprLeftRes = left.evaluate(value);
+    ExpressionResult exprRightRes = right.evaluate(value);
+    ExpressionResult val1 = exprLeftRes;
+    if (exprLeftRes.isNull() || exprRightRes.isNull()) {
+      exprLeftRes.set(DataType.BooleanType, false);
+      return exprLeftRes;
+    }
+    if (exprLeftRes.getDataType() != exprRightRes.getDataType()) {
+      if (exprLeftRes.getDataType().getPresedenceOrder() < exprRightRes.getDataType()
+          .getPresedenceOrder()) {
+        val1 = exprRightRes;
+      }
+
+    }
+    boolean result = false;
+    switch (val1.getDataType()) {
+      case StringType:
+        result = exprLeftRes.getString().compareTo(exprRightRes.getString()) > 0;
+        break;
+      case DoubleType:
+        result = exprLeftRes.getDouble() > (exprRightRes.getDouble());
+        break;
+      case IntegerType:
+        result = exprLeftRes.getInt() > (exprRightRes.getInt());
+        break;
+      case TimestampType:
+        result = exprLeftRes.getTime() > (exprRightRes.getTime());
+        break;
+      case LongType:
+        result = exprLeftRes.getLong() > (exprRightRes.getLong());
+        break;
+      case DecimalType:
+        result = exprLeftRes.getDecimal().compareTo(exprRightRes.getDecimal()) > 0;
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    }
+    val1.set(DataType.BooleanType, result);
+    return val1;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.GREATERTHAN;
+  }
+
+  @Override public String getString() {
+    return "GreaterThan(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java
new file mode 100644
index 0000000..6574d3e
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.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.carbondata.scan.expression.conditional;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class InExpression extends BinaryConditionalExpression {
+  private static final long serialVersionUID = -3149927446694175489L;
+
+  protected transient Set<ExpressionResult> setOfExprResult;
+
+  public InExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult leftRsult = left.evaluate(value);
+
+    if (setOfExprResult == null) {
+      ExpressionResult rightRsult = right.evaluate(value);
+      ExpressionResult val = null;
+      setOfExprResult = new HashSet<ExpressionResult>(10);
+      for (ExpressionResult expressionResVal : rightRsult.getList()) {
+
+        if (leftRsult.getDataType().name().equals(expressionResVal.getDataType().name())) {
+          if (expressionResVal.getDataType().getPresedenceOrder() < leftRsult.getDataType()
+              .getPresedenceOrder()) {
+            val = leftRsult;
+          } else {
+            val = expressionResVal;
+          }
+
+          switch (val.getDataType()) {
+            case StringType:
+              val = new ExpressionResult(val.getDataType(), expressionResVal.getString());
+              break;
+            case IntegerType:
+              val = new ExpressionResult(val.getDataType(), expressionResVal.getInt());
+              break;
+            case DoubleType:
+              val = new ExpressionResult(val.getDataType(), expressionResVal.getDouble());
+              break;
+            case TimestampType:
+              val = new ExpressionResult(val.getDataType(), expressionResVal.getTime());
+              break;
+            case LongType:
+              val = new ExpressionResult(val.getDataType(), expressionResVal.getLong());
+              break;
+            case DecimalType:
+              val = new ExpressionResult(val.getDataType(), expressionResVal.getDecimal());
+              break;
+            default:
+              throw new FilterUnsupportedException(
+                  "DataType: " + val.getDataType() + " not supported for the filter expression");
+          }
+
+        }
+        setOfExprResult.add(val);
+
+      }
+    }
+    leftRsult.set(DataType.BooleanType, setOfExprResult.contains(leftRsult));
+    return leftRsult;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.IN;
+  }
+
+  @Override public String getString() {
+    return "IN(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java
new file mode 100644
index 0000000..f194157
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.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.carbondata.scan.expression.conditional;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class LessThanEqualToExpression extends BinaryConditionalExpression {
+  private static final long serialVersionUID = 1L;
+
+  public LessThanEqualToExpression(Expression left, Expression right) {
+    super(left, right);
+    // TODO Auto-generated constructor stub
+  }
+
+  public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult elRes = left.evaluate(value);
+    ExpressionResult erRes = right.evaluate(value);
+    ExpressionResult exprResValue1 = elRes;
+    if (elRes.isNull() || erRes.isNull()) {
+      elRes.set(DataType.BooleanType, false);
+      return elRes;
+    }
+    if (elRes.getDataType() != erRes.getDataType()) {
+      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
+        exprResValue1 = erRes;
+      }
+
+    }
+    boolean result = false;
+    switch (exprResValue1.getDataType()) {
+      case StringType:
+        result = elRes.getString().compareTo(erRes.getString()) <= 0;
+        break;
+      case IntegerType:
+        result = elRes.getInt() <= (erRes.getInt());
+        break;
+      case DoubleType:
+        result = elRes.getDouble() <= (erRes.getDouble());
+        break;
+      case TimestampType:
+        result = elRes.getTime() <= (erRes.getTime());
+        break;
+      case LongType:
+        result = elRes.getLong() <= (erRes.getLong());
+        break;
+      case DecimalType:
+        result = elRes.getDecimal().compareTo(erRes.getDecimal()) <= 0;
+        break;
+      default:
+        throw new FilterUnsupportedException("DataType: " + exprResValue1.getDataType()
+            + " not supported for the filter expression");
+    }
+    exprResValue1.set(DataType.BooleanType, result);
+    return exprResValue1;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    // TODO Auto-generated method stub
+    return ExpressionType.LESSTHAN_EQUALTO;
+  }
+
+  @Override public String getString() {
+    return "LessThanEqualTo(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
new file mode 100644
index 0000000..b55d0cc
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
@@ -0,0 +1,90 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.conditional;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class LessThanExpression extends BinaryConditionalExpression {
+
+  private static final long serialVersionUID = 6343040416663699924L;
+
+  public LessThanExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult erRes = right.evaluate(value);
+    ExpressionResult elRes = left.evaluate(value);
+
+    ExpressionResult val1 = elRes;
+
+    boolean result = false;
+
+    if (elRes.isNull() || erRes.isNull()) {
+      elRes.set(DataType.BooleanType, false);
+      return elRes;
+    }
+    if (elRes.getDataType() != erRes.getDataType()) {
+      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
+        val1 = erRes;
+      }
+
+    }
+    switch (val1.getDataType()) {
+      case StringType:
+        result = elRes.getString().compareTo(erRes.getString()) < 0;
+        break;
+      case IntegerType:
+        result = elRes.getInt() < (erRes.getInt());
+        break;
+      case DoubleType:
+        result = elRes.getDouble() < (erRes.getDouble());
+        break;
+      case TimestampType:
+        result = elRes.getTime() < (erRes.getTime());
+        break;
+      case LongType:
+        result = elRes.getLong() < (erRes.getLong());
+        break;
+      case DecimalType:
+        result = elRes.getDecimal().compareTo(erRes.getDecimal()) < 0;
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    }
+    val1.set(DataType.BooleanType, result);
+    return val1;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.LESSTHAN;
+  }
+
+  @Override public String getString() {
+    return "LessThan(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java
new file mode 100644
index 0000000..b252793
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.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.carbondata.scan.expression.conditional;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class ListExpression extends Expression {
+  private static final long serialVersionUID = 1L;
+
+  public ListExpression(List<Expression> children) {
+    this.children = children;
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    List<ExpressionResult> listOfExprRes = new ArrayList<ExpressionResult>(10);
+
+    for (Expression expr : children) {
+      listOfExprRes.add(expr.evaluate(value));
+    }
+    return new ExpressionResult(listOfExprRes);
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    // TODO Auto-generated method stub
+    return ExpressionType.LIST;
+  }
+
+  @Override public String getString() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
new file mode 100644
index 0000000..df40f7f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
@@ -0,0 +1,93 @@
+/*
+ * 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.carbondata.scan.expression.conditional;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class NotEqualsExpression extends BinaryConditionalExpression {
+
+  private static final long serialVersionUID = 8684006025540863973L;
+
+  public NotEqualsExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult elRes = left.evaluate(value);
+    ExpressionResult erRes = right.evaluate(value);
+
+    boolean result = false;
+    ExpressionResult val1 = elRes;
+    ExpressionResult val2 = erRes;
+
+    if (elRes.isNull() || erRes.isNull()) {
+      result = elRes.isNull() != erRes.isNull();
+      val1.set(DataType.BooleanType, result);
+      return val1;
+    }
+
+    //default implementation if the data types are different for the resultsets
+    if (elRes.getDataType() != erRes.getDataType()) {
+      //            result = elRes.getString().equals(erRes.getString());
+      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
+        val1 = erRes;
+        val2 = elRes;
+      }
+    }
+    switch (val1.getDataType()) {
+      case StringType:
+        result = !val1.getString().equals(val2.getString());
+        break;
+      case IntegerType:
+        result = val1.getInt().intValue() != val2.getInt().intValue();
+        break;
+      case DoubleType:
+        result = val1.getDouble().doubleValue() != val2.getDouble().doubleValue();
+        break;
+      case TimestampType:
+        result = val1.getTime().longValue() != val2.getTime().longValue();
+        break;
+      case LongType:
+        result = elRes.getLong().longValue() != (erRes.getLong()).longValue();
+        break;
+      case DecimalType:
+        result = elRes.getDecimal().compareTo(erRes.getDecimal()) != 0;
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "DataType: " + val1.getDataType() + " not supported for the filter expression");
+    }
+    val1.set(DataType.BooleanType, result);
+    return val1;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.NOT_EQUALS;
+  }
+
+  @Override public String getString() {
+    return "NotEquals(" + left.getString() + ',' + right.getString() + ')';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java b/core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
new file mode 100644
index 0000000..5777c4d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
@@ -0,0 +1,100 @@
+/*
+ * 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.carbondata.scan.expression.conditional;
+
+import java.util.HashSet;
+import java.util.Set;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class NotInExpression extends BinaryConditionalExpression {
+  private static final long serialVersionUID = -6835841923752118034L;
+  protected transient Set<ExpressionResult> setOfExprResult;
+
+  public NotInExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult leftRsult = left.evaluate(value);
+
+    if (setOfExprResult == null) {
+      ExpressionResult val = null;
+
+      ExpressionResult rightRsult = right.evaluate(value);
+      setOfExprResult = new HashSet<ExpressionResult>(10);
+      for (ExpressionResult exprResVal : rightRsult.getList()) {
+
+        if (leftRsult.getDataType().name().equals(exprResVal.getDataType().name())) {
+          if (exprResVal.getDataType().getPresedenceOrder() < leftRsult.getDataType()
+              .getPresedenceOrder()) {
+            val = leftRsult;
+          } else {
+            val = exprResVal;
+          }
+
+          switch (val.getDataType()) {
+            case StringType:
+              val = new ExpressionResult(val.getDataType(), exprResVal.getString());
+              break;
+            case IntegerType:
+              val = new ExpressionResult(val.getDataType(), exprResVal.getInt());
+              break;
+            case DoubleType:
+              val = new ExpressionResult(val.getDataType(), exprResVal.getDouble());
+              break;
+            case TimestampType:
+              val = new ExpressionResult(val.getDataType(), exprResVal.getTime());
+              break;
+            case LongType:
+              val = new ExpressionResult(val.getDataType(), exprResVal.getLong());
+              break;
+            case DecimalType:
+              val = new ExpressionResult(val.getDataType(), exprResVal.getDecimal());
+              break;
+            default:
+              throw new FilterUnsupportedException(
+                  "DataType: " + val.getDataType() + " not supported for the filter expression");
+          }
+
+        }
+        setOfExprResult.add(val);
+
+      }
+    }
+    leftRsult.set(DataType.BooleanType, !setOfExprResult.contains(leftRsult));
+
+    return leftRsult;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.NOT_IN;
+  }
+
+  @Override public String getString() {
+    return "NOT IN(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java b/core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
new file mode 100644
index 0000000..c86a38d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
@@ -0,0 +1,93 @@
+/*
+ * 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.carbondata.scan.expression.exception;
+
+import java.util.Locale;
+
+public class FilterUnsupportedException extends Exception {
+
+  /**
+   * default serial version ID.
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * The Error message.
+   */
+  private String msg = "";
+
+  /**
+   * Constructor
+   *
+   * @param errorCode The error code for this exception.
+   * @param msg       The error message for this exception.
+   */
+  public FilterUnsupportedException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param errorCode The error code for this exception.
+   * @param msg       The error message for this exception.
+   */
+  public FilterUnsupportedException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param errorCode The error code for this exception.
+   * @param msg       The error message for this exception.
+   */
+  public FilterUnsupportedException(Throwable t) {
+    super(t);
+  }
+
+  /**
+   * This method is used to get the localized message.
+   *
+   * @param locale - A Locale object represents a specific geographical,
+   *               political, or cultural region.
+   * @return - Localized error message.
+   */
+  public String getLocalizedMessage(Locale locale) {
+    return "";
+  }
+
+  /**
+   * getLocalizedMessage
+   */
+  @Override public String getLocalizedMessage() {
+    return super.getLocalizedMessage();
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java b/core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
new file mode 100644
index 0000000..b3f63bd
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
@@ -0,0 +1,63 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.logical;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class AndExpression extends BinaryLogicalExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  public AndExpression(Expression left, Expression right) {
+    super(left, right);
+    // TODO Auto-generated constructor stub
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult resultLeft = left.evaluate(value);
+    ExpressionResult resultRight = right.evaluate(value);
+    switch (resultLeft.getDataType()) {
+      case BooleanType:
+        resultLeft.set(DataType.BooleanType, (resultLeft.getBoolean() && resultRight.getBoolean()));
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "Incompatible datatype for applying AND Expression Filter");
+    }
+
+    return resultLeft;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    // TODO Auto-generated method stub
+    return ExpressionType.AND;
+  }
+
+  @Override public String getString() {
+    // TODO Auto-generated method stub
+    return "And(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/logical/BinaryLogicalExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/logical/BinaryLogicalExpression.java b/core/src/main/java/org/carbondata/scan/expression/logical/BinaryLogicalExpression.java
new file mode 100644
index 0000000..63cef25
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/logical/BinaryLogicalExpression.java
@@ -0,0 +1,127 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.logical;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.scan.expression.BinaryExpression;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.LiteralExpression;
+
+public abstract class BinaryLogicalExpression extends BinaryExpression {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+
+  public BinaryLogicalExpression(Expression left, Expression right) {
+    super(left, right);
+    // TODO Auto-generated constructor stub
+  }
+
+  public List<ExpressionResult> getLiterals() {
+    List<ExpressionResult> listOfExp =
+        new ArrayList<ExpressionResult>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    getExpressionResultList(this, listOfExp);
+    Collections.sort(listOfExp);
+    return listOfExp;
+  }
+
+  // Will get the column informations involved in the expressions by
+  // traversing the tree
+  public List<ColumnExpression> getColumnList() {
+    // TODO
+    List<ColumnExpression> listOfExp =
+        new ArrayList<ColumnExpression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    getColumnList(this, listOfExp);
+    return listOfExp;
+  }
+
+  private void getColumnList(Expression expression, List<ColumnExpression> lst) {
+    if (expression instanceof ColumnExpression) {
+      ColumnExpression colExp = (ColumnExpression) expression;
+      boolean found = false;
+
+      for (ColumnExpression currentColExp : lst) {
+        if (currentColExp.getColumnName().equals(colExp.getColumnName())) {
+          found = true;
+          colExp.setColIndex(currentColExp.getColIndex());
+          break;
+        }
+      }
+      if (!found) {
+        colExp.setColIndex(lst.size());
+        lst.add(colExp);
+      }
+    }
+    for (Expression child : expression.getChildren()) {
+      getColumnList(child, lst);
+    }
+  }
+
+  public boolean isSingleDimension() {
+    List<ColumnExpression> listOfExp =
+        new ArrayList<ColumnExpression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    getColumnList(this, listOfExp);
+    if (listOfExp.size() == 1 && listOfExp.get(0).isDimension()) {
+      return true;
+    }
+    return false;
+
+  }
+
+  private void getExpressionResultList(Expression binaryConditionalExpression,
+      List<ExpressionResult> listOfExp) {
+    if (binaryConditionalExpression instanceof LiteralExpression) {
+      ExpressionResult colExp =
+          ((LiteralExpression) binaryConditionalExpression).getExpressionResult();
+      listOfExp.add(colExp);
+    }
+    for (Expression child : binaryConditionalExpression.getChildren()) {
+      getExpressionResultList(child, listOfExp);
+    }
+
+  }
+
+  /**
+   * the method will return flag (true or false) depending on the existence of the
+   * direct dictionary columns in conditional expression
+   *
+   * @return the method will return flag (true or false)
+   */
+  public boolean isDirectDictionaryColumns() {
+    List<ColumnExpression> listOfExp =
+        new ArrayList<ColumnExpression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    getColumnList(this, listOfExp);
+    for (ColumnExpression ce : listOfExp) {
+      if (!ce.getCarbonColumn().hasEncoding(Encoding.DICTIONARY)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java b/core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java
new file mode 100644
index 0000000..239af8f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.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.carbondata.scan.expression.logical;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.UnaryExpression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class NotExpression extends UnaryExpression {
+  private static final long serialVersionUID = 1L;
+
+  public NotExpression(Expression child) {
+    super(child);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult expResult = child.evaluate(value);
+    expResult.set(DataType.BooleanType, !(expResult.getBoolean()));
+    switch (expResult.getDataType()) {
+      case BooleanType:
+        expResult.set(DataType.BooleanType, !(expResult.getBoolean()));
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "Incompatible datatype for applying NOT Expression Filter");
+    }
+    return expResult;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.NOT;
+  }
+
+  @Override public String getString() {
+    return "Not(" + child.getString() + ')';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java b/core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
new file mode 100644
index 0000000..86d6c8d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
@@ -0,0 +1,60 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.logical;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class OrExpression extends BinaryLogicalExpression {
+
+  private static final long serialVersionUID = 4220598043176438380L;
+
+  public OrExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult resultLeft = left.evaluate(value);
+    ExpressionResult resultRight = right.evaluate(value);
+    switch (resultLeft.getDataType()) {
+      case BooleanType:
+        resultLeft.set(DataType.BooleanType, (resultLeft.getBoolean() || resultRight.getBoolean()));
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "Incompatible datatype for applying OR Expression Filter");
+    }
+
+    return resultLeft;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.OR;
+  }
+
+  @Override public String getString() {
+    return "Or(" + left.getString() + ',' + right.getString() + ')';
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/DimColumnFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/DimColumnFilterInfo.java b/core/src/main/java/org/carbondata/scan/filter/DimColumnFilterInfo.java
new file mode 100644
index 0000000..e3fd3aa
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/DimColumnFilterInfo.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.filter;
+
+import java.io.Serializable;
+import java.util.List;
+
+public class DimColumnFilterInfo implements Serializable {
+
+  private static final long serialVersionUID = 8181578747306832771L;
+
+  private boolean isIncludeFilter;
+
+  private List<Integer> filterList;
+
+  /**
+   * maintain the no dictionary filter values list.
+   */
+  private List<byte[]> noDictionaryFilterValuesList;
+
+  public List<byte[]> getNoDictionaryFilterValuesList() {
+    return noDictionaryFilterValuesList;
+  }
+
+  public boolean isIncludeFilter() {
+    return isIncludeFilter;
+  }
+
+  public void setIncludeFilter(boolean isIncludeFilter) {
+    this.isIncludeFilter = isIncludeFilter;
+  }
+
+  public List<Integer> getFilterList() {
+    return filterList;
+  }
+
+  public void setFilterList(List<Integer> filterList) {
+    this.filterList = filterList;
+  }
+
+  public void setFilterListForNoDictionaryCols(List<byte[]> noDictionaryFilterValuesList) {
+    this.noDictionaryFilterValuesList = noDictionaryFilterValuesList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java b/core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java
new file mode 100644
index 0000000..e5dd7b4
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java
@@ -0,0 +1,360 @@
+/*
+ * 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.carbondata.scan.filter;
+
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.carbon.datastore.DataRefNodeFinder;
+import org.carbondata.core.carbon.datastore.IndexKey;
+import org.carbondata.core.carbon.datastore.block.AbstractIndex;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.BinaryExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.conditional.BinaryConditionalExpression;
+import org.carbondata.scan.expression.conditional.ConditionalExpression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.expression.logical.BinaryLogicalExpression;
+import org.carbondata.scan.filter.executer.FilterExecuter;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.resolver.ConditionalFilterResolverImpl;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+import org.carbondata.scan.filter.resolver.LogicalFilterResolverImpl;
+import org.carbondata.scan.filter.resolver.RowLevelFilterResolverImpl;
+import org.carbondata.scan.filter.resolver.RowLevelRangeFilterResolverImpl;
+
+public class FilterExpressionProcessor implements FilterProcessor {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(FilterExpressionProcessor.class.getName());
+
+  /**
+   * Implementation will provide the resolved form of filters based on the
+   * filter expression tree which is been passed in Expression instance.
+   *
+   * @param expressionTree  , filter expression tree
+   * @param tableIdentifier ,contains carbon store informations
+   * @return a filter resolver tree
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  public FilterResolverIntf getFilterResolver(Expression expressionTree,
+      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException {
+    if (null != expressionTree && null != tableIdentifier) {
+      return getFilterResolvertree(expressionTree, tableIdentifier);
+    }
+    return null;
+  }
+
+  /**
+   * This API will scan the Segment level all btrees and selects the required
+   * block reference  nodes inorder to push the same to executer for applying filters
+   * on the respective data reference node.
+   * Following Algorithm is followed in below API
+   * Step:1 Get the start end key based on the filter tree resolver information
+   * Step:2 Prepare the IndexKeys inorder to scan the tree and get the start and end reference
+   * node(block)
+   * Step:3 Once data reference node ranges retrieved traverse the node within this range
+   * and select the node based on the block min and max value and the filter value.
+   * Step:4 The selected blocks will be send to executers for applying the filters with the help
+   * of Filter executers.
+   *
+   * @throws QueryExecutionException
+   */
+  public List<DataRefNode> getFilterredBlocks(DataRefNode btreeNode,
+      FilterResolverIntf filterResolver, AbstractIndex tableSegment,
+      AbsoluteTableIdentifier tableIdentifier) throws QueryExecutionException {
+    // Need to get the current dimension tables
+    List<DataRefNode> listOfDataBlocksToScan = new ArrayList<DataRefNode>();
+    // getting the start and end index key based on filter for hitting the
+    // selected block reference nodes based on filter resolver tree.
+    LOGGER.info("preparing the start and end key for finding"
+        + "start and end block as per filter resolver");
+    List<IndexKey> listOfStartEndKeys = new ArrayList<IndexKey>(2);
+    FilterUtil.traverseResolverTreeAndGetStartAndEndKey(tableSegment.getSegmentProperties(),
+        tableIdentifier, filterResolver, listOfStartEndKeys);
+    // reading the first value from list which has start key
+    IndexKey searchStartKey = listOfStartEndKeys.get(0);
+    // reading the last value from list which has end key
+    IndexKey searchEndKey = listOfStartEndKeys.get(1);
+    if (null == searchStartKey && null == searchEndKey) {
+      try {
+        // TODO need to handle for no dictionary dimensions
+        searchStartKey =
+            FilterUtil.prepareDefaultStartIndexKey(tableSegment.getSegmentProperties());
+        // TODO need to handle for no dictionary dimensions
+        searchEndKey = FilterUtil.prepareDefaultEndIndexKey(tableSegment.getSegmentProperties());
+      } catch (KeyGenException e) {
+        return listOfDataBlocksToScan;
+      }
+    }
+
+    LOGGER.info("Successfully retrieved the start and end key");
+    long startTimeInMillis = System.currentTimeMillis();
+    DataRefNodeFinder blockFinder = new BTreeDataRefNodeFinder(
+        tableSegment.getSegmentProperties().getDimensionColumnsValueSize());
+    DataRefNode startBlock = blockFinder.findFirstDataBlock(btreeNode, searchStartKey);
+    DataRefNode endBlock = blockFinder.findLastDataBlock(btreeNode, searchEndKey);
+    FilterExecuter filterExecuter =
+            FilterUtil.getFilterExecuterTree(filterResolver, tableSegment.getSegmentProperties());
+    while (startBlock != endBlock) {
+      addBlockBasedOnMinMaxValue(filterExecuter, listOfDataBlocksToScan, startBlock,
+          tableSegment.getSegmentProperties());
+      startBlock = startBlock.getNextDataRefNode();
+    }
+    addBlockBasedOnMinMaxValue(filterExecuter, listOfDataBlocksToScan, endBlock,
+        tableSegment.getSegmentProperties());
+    LOGGER.info("Total Time in retrieving the data reference node" + "after scanning the btree " + (
+        System.currentTimeMillis() - startTimeInMillis)
+        + " Total number of data reference node for executing filter(s) " + listOfDataBlocksToScan
+        .size());
+
+    return listOfDataBlocksToScan;
+  }
+
+  /**
+   * Selects the blocks based on col max and min value.
+   *
+   * @param filterResolver
+   * @param listOfDataBlocksToScan
+   * @param dataRefNode
+   * @param segmentProperties
+   */
+  private void addBlockBasedOnMinMaxValue(FilterExecuter filterExecuter,
+      List<DataRefNode> listOfDataBlocksToScan, DataRefNode dataRefNode,
+      SegmentProperties segmentProperties) {
+
+    BitSet bitSet = filterExecuter
+        .isScanRequired(dataRefNode.getColumnsMaxValue(), dataRefNode.getColumnsMinValue());
+    if (!bitSet.isEmpty()) {
+      listOfDataBlocksToScan.add(dataRefNode);
+
+    }
+  }
+
+  /**
+   * API will return a filter resolver instance which will be used by
+   * executers to evaluate or execute the filters.
+   *
+   * @param expressionTree , resolver tree which will hold the resolver tree based on
+   *                       filter expression.
+   * @return FilterResolverIntf type.
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  private FilterResolverIntf getFilterResolvertree(Expression expressionTree,
+      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException {
+    FilterResolverIntf filterEvaluatorTree =
+        createFilterResolverTree(expressionTree, tableIdentifier, null);
+    traverseAndResolveTree(filterEvaluatorTree, tableIdentifier);
+    return filterEvaluatorTree;
+  }
+
+  /**
+   * constructing the filter resolver tree based on filter expression.
+   * this method will visit each node of the filter resolver and prepares
+   * the surrogates of the filter members which are involved filter
+   * expression.
+   *
+   * @param filterResolverTree
+   * @param tableIdentifier
+   * @throws QueryExecutionException
+   */
+  private void traverseAndResolveTree(FilterResolverIntf filterResolverTree,
+      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException {
+    if (null == filterResolverTree) {
+      return;
+    }
+    traverseAndResolveTree(filterResolverTree.getLeft(), tableIdentifier);
+
+    filterResolverTree.resolve(tableIdentifier);
+
+    traverseAndResolveTree(filterResolverTree.getRight(), tableIdentifier);
+  }
+
+  /**
+   * Pattern used : Visitor Pattern
+   * Method will create filter resolver tree based on the filter expression tree,
+   * in this algorithm based on the expression instance the resolvers will created
+   *
+   * @param expressionTree
+   * @param tableIdentifier
+   * @return
+   */
+  private FilterResolverIntf createFilterResolverTree(Expression expressionTree,
+      AbsoluteTableIdentifier tableIdentifier, Expression intermediateExpression) {
+    ExpressionType filterExpressionType = expressionTree.getFilterExpressionType();
+    BinaryExpression currentExpression = null;
+    BinaryLogicalExpression logicalExpression = null;
+    switch (filterExpressionType) {
+      case OR:
+        currentExpression = (BinaryExpression) expressionTree;
+        return new LogicalFilterResolverImpl(
+            createFilterResolverTree(currentExpression.getLeft(), tableIdentifier,
+                currentExpression),
+            createFilterResolverTree(currentExpression.getRight(), tableIdentifier,
+                currentExpression), filterExpressionType);
+      case AND:
+        logicalExpression = (BinaryLogicalExpression) expressionTree;
+        return new LogicalFilterResolverImpl(
+            createFilterResolverTree(logicalExpression.getLeft(), tableIdentifier,
+                currentExpression),
+            createFilterResolverTree(logicalExpression.getRight(), tableIdentifier,
+                currentExpression), filterExpressionType);
+      case EQUALS:
+      case IN:
+        return getFilterResolverBasedOnExpressionType(ExpressionType.EQUALS, false, expressionTree,
+            tableIdentifier, expressionTree);
+      case GREATERTHAN:
+      case GREATERTHAN_EQUALTO:
+      case LESSTHAN:
+      case LESSTHAN_EQUALTO:
+        return getFilterResolverBasedOnExpressionType(ExpressionType.EQUALS, true, expressionTree,
+            tableIdentifier, expressionTree);
+
+      case NOT_EQUALS:
+      case NOT_IN:
+        return getFilterResolverBasedOnExpressionType(ExpressionType.NOT_EQUALS, false,
+            expressionTree, tableIdentifier, expressionTree);
+
+      default:
+        return getFilterResolverBasedOnExpressionType(ExpressionType.UNKNOWN, false, expressionTree,
+            tableIdentifier, expressionTree);
+    }
+  }
+
+  /**
+   * Factory method which will return the resolver instance based on filter expression
+   * expressions.
+   */
+  private FilterResolverIntf getFilterResolverBasedOnExpressionType(
+      ExpressionType filterExpressionType, boolean isExpressionResolve, Expression expression,
+      AbsoluteTableIdentifier tableIdentifier, Expression expressionTree) {
+    BinaryConditionalExpression currentCondExpression = null;
+    ConditionalExpression condExpression = null;
+    switch (filterExpressionType) {
+      case EQUALS:
+        currentCondExpression = (BinaryConditionalExpression) expression;
+        if (currentCondExpression.isSingleDimension()
+            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
+            != DataType.ARRAY
+            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
+            != DataType.STRUCT) {
+          // getting new dim index.
+          if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
+              .hasEncoding(Encoding.DICTIONARY) || currentCondExpression.getColumnList().get(0)
+              .getCarbonColumn().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+            if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
+                && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight()) || (
+                FilterUtil.checkIfExpressionContainsUnknownExp(currentCondExpression.getRight())
+                    || FilterUtil
+                    .checkIfExpressionContainsUnknownExp(currentCondExpression.getLeft()))) {
+              return new RowLevelFilterResolverImpl(expression, isExpressionResolve, true,
+                  tableIdentifier);
+            }
+            if (currentCondExpression.getFilterExpressionType() == ExpressionType.GREATERTHAN
+                || currentCondExpression.getFilterExpressionType() == ExpressionType.LESSTHAN
+                || currentCondExpression.getFilterExpressionType()
+                == ExpressionType.GREATERTHAN_EQUALTO
+                || currentCondExpression.getFilterExpressionType()
+                == ExpressionType.LESSTHAN_EQUALTO) {
+              if (currentCondExpression.getColumnList().get(0).getCarbonColumn()
+                  .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+                return new RowLevelFilterResolverImpl(expression, isExpressionResolve, true,
+                    tableIdentifier);
+              }
+              return new RowLevelRangeFilterResolverImpl(expression, isExpressionResolve, true,
+                  tableIdentifier);
+            }
+          }
+          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, true);
+
+        }
+        break;
+      case NOT_EQUALS:
+        currentCondExpression = (BinaryConditionalExpression) expression;
+        if (currentCondExpression.isSingleDimension()
+            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
+            != DataType.ARRAY
+            && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
+            != DataType.STRUCT) {
+          if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
+              .hasEncoding(Encoding.DICTIONARY)) {
+            if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
+                && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight())
+                || (FilterUtil
+                    .checkIfExpressionContainsUnknownExp(currentCondExpression.getRight())
+                || FilterUtil
+                    .checkIfExpressionContainsUnknownExp(currentCondExpression.getLeft()))) {
+              return new RowLevelFilterResolverImpl(expression, isExpressionResolve, false,
+                  tableIdentifier);
+            }
+            if (expressionTree.getFilterExpressionType() == ExpressionType.GREATERTHAN
+                || expressionTree.getFilterExpressionType() == ExpressionType.LESSTHAN
+                || expressionTree.getFilterExpressionType() == ExpressionType.GREATERTHAN_EQUALTO
+                || expressionTree.getFilterExpressionType() == ExpressionType.LESSTHAN_EQUALTO) {
+
+              return new RowLevelRangeFilterResolverImpl(expression, isExpressionResolve, false,
+                  tableIdentifier);
+            }
+
+            return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false);
+          }
+          return new ConditionalFilterResolverImpl(expression, isExpressionResolve, false);
+        }
+        break;
+      default:
+        condExpression = (ConditionalExpression) expression;
+        if (condExpression.isSingleDimension()
+            && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
+            != DataType.ARRAY
+            && condExpression.getColumnList().get(0).getCarbonColumn().getDataType()
+            != DataType.STRUCT) {
+          condExpression = (ConditionalExpression) expression;
+          if (condExpression.isSingleDimension()) {
+            if (!condExpression.getColumnList().get(0).getCarbonColumn()
+                .hasEncoding(Encoding.DICTIONARY)) {
+              if (FilterUtil.checkIfExpressionContainsColumn(expression)) {
+                return new RowLevelFilterResolverImpl(expression, isExpressionResolve, false,
+                    tableIdentifier);
+              } else if (expressionTree.getFilterExpressionType() == ExpressionType.UNKNOWN) {
+                return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
+              }
+
+              return new ConditionalFilterResolverImpl(expression, true, true);
+            }
+          }
+        } else {
+          return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
+        }
+    }
+    return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/FilterProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/FilterProcessor.java b/core/src/main/java/org/carbondata/scan/filter/FilterProcessor.java
new file mode 100644
index 0000000..3531621
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/FilterProcessor.java
@@ -0,0 +1,60 @@
+package org.carbondata.scan.filter;
+
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+import java.util.List;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.carbon.datastore.block.AbstractIndex;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+
+public interface FilterProcessor {
+
+  /**
+   * API will provide the resolved form of filters based on the filter
+   * expression tree which is been passed.
+   *
+   * @param expressionTree  , filter expression tree
+   * @param tableIdentifier ,contains carbon store informations.
+   * @return
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  FilterResolverIntf getFilterResolver(Expression expressionTree,
+      AbsoluteTableIdentifier tableIdentifier) throws FilterUnsupportedException;
+
+  /**
+   * This API is exposed inorder to get the required block reference node
+   * based on the filter.The block list will be send to the executer tasks inorder
+   * to apply filters.
+   *
+   * @param filterResolver DataBlock list with resolved filters
+   * @return list of DataRefNode.
+   * @throws QueryExecutionException
+   */
+  List<DataRefNode> getFilterredBlocks(DataRefNode dataRefNode, FilterResolverIntf filterResolver,
+      AbstractIndex segmentIndexBuilder, AbsoluteTableIdentifier tableIdentifier)
+      throws QueryExecutionException;
+
+}


[34/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
deleted file mode 100644
index 4897736..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/ConditionalFilterResolverImpl.java
+++ /dev/null
@@ -1,239 +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.carbondata.query.filter.resolver;
-
-import java.util.List;
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.visitor.FilterInfoTypeVisitorFactory;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class ConditionalFilterResolverImpl implements FilterResolverIntf {
-
-  private static final long serialVersionUID = 1838955268462201691L;
-  protected Expression exp;
-  protected boolean isExpressionResolve;
-  protected boolean isIncludeFilter;
-  private DimColumnResolvedFilterInfo dimColResolvedFilterInfo;
-
-  public ConditionalFilterResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter) {
-    this.exp = exp;
-    this.isExpressionResolve = isExpressionResolve;
-    this.isIncludeFilter = isIncludeFilter;
-    this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo();
-  }
-
-  /**
-   * This API will resolve the filter expression and generates the
-   * dictionaries for executing/evaluating the filter expressions in the
-   * executer layer.
-   *
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
-      throws FilterUnsupportedException {
-    FilterResolverMetadata metadata = new FilterResolverMetadata();
-    metadata.setTableIdentifier(absoluteTableIdentifier);
-    if ((!isExpressionResolve) && exp instanceof BinaryConditionalExpression) {
-      BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression) exp;
-      Expression leftExp = binaryConditionalExpression.getLeft();
-      Expression rightExp = binaryConditionalExpression.getRight();
-      if (leftExp instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) leftExp;
-        metadata.setColumnExpression(columnExpression);
-        metadata.setExpression(rightExp);
-        metadata.setIncludeFilter(isIncludeFilter);
-        // If imei=imei comes in filter condition then we need to
-        // skip processing of right expression.
-        // This flow has reached here assuming that this is a single
-        // column expression.
-        // we need to check if the other expression contains column
-        // expression or not in depth.
-        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)||
-            FilterUtil.isExpressionNeedsToResolved(rightExp,isIncludeFilter) &&
-            columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)){
-          isExpressionResolve = true;
-        } else {
-          //Visitor pattern is been used in this scenario inorder to populate the
-          // dimColResolvedFilterInfo
-          //visitable object with filter member values based on the visitor type, currently there
-          //3 types of visitors custom,direct and no dictionary, all types of visitor populate
-          //the visitable instance as per its buisness logic which is different for all the
-          // visitors.
-          dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
-              FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
-              metadata);
-        }
-      } else if (rightExp instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) rightExp;
-        metadata.setColumnExpression(columnExpression);
-        metadata.setExpression(leftExp);
-        metadata.setIncludeFilter(isIncludeFilter);
-        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
-          isExpressionResolve = true;
-        } else {
-          // if imei=imei comes in filter condition then we need to
-          // skip processing of right expression.
-          // This flow has reached here assuming that this is a single
-          // column expression.
-          // we need to check if the other expression contains column
-          // expression or not in depth.
-          if (FilterUtil.checkIfExpressionContainsColumn(leftExp)) {
-            isExpressionResolve = true;
-          } else {
-
-            dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
-                FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
-                metadata);
-
-          }
-        }
-      } else {
-        isExpressionResolve = true;
-      }
-    }
-    if (isExpressionResolve && exp instanceof ConditionalExpression) {
-      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
-      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
-      metadata.setColumnExpression(columnList.get(0));
-      metadata.setExpression(exp);
-      metadata.setIncludeFilter(isIncludeFilter);
-      if (!columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) || columnList.get(0)
-          .getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
-            FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnList.get(0)), metadata);
-
-      } else if (columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) && !(
-          columnList.get(0).getDimension().getDataType()
-              == org.carbondata.core.carbon.metadata.datatype.DataType.STRUCT
-              || columnList.get(0).getDimension().getDataType()
-              == org.carbondata.core.carbon.metadata.datatype.DataType.ARRAY)) {
-        dimColResolvedFilterInfo.setFilterValues(FilterUtil
-            .getFilterListForAllValues(absoluteTableIdentifier, exp, columnList.get(0),
-                isIncludeFilter));
-
-        dimColResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
-        dimColResolvedFilterInfo.setDimension(columnList.get(0).getDimension());
-      }
-    }
-
-  }
-
-  /**
-   * Left node will not be presentin this scenario
-   *
-   * @return left node of type FilterResolverIntf instance
-   */
-  public FilterResolverIntf getLeft() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  /**
-   * Right node will not be presentin this scenario
-   *
-   * @return left node of type FilterResolverIntf instance
-   */
-  @Override public FilterResolverIntf getRight() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  /**
-   * Method will return the DimColumnResolvedFilterInfo instance which consists
-   * the mapping of the respective dimension and its surrogates involved in
-   * filter expression.
-   *
-   * @return DimColumnResolvedFilterInfo
-   */
-  public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
-    return dimColResolvedFilterInfo;
-  }
-
-  /**
-   * method will calculates the start key based on the filter surrogates
-   */
-  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
-    if (null == dimColResolvedFilterInfo.getStarIndexKey()) {
-      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-          setOfStartKeyByteArray);
-    }
-  }
-
-  /**
-   * method will get the start key based on the filter surrogates
-   *
-   * @return end IndexKey
-   */
-  @Override public void getEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
-    if (null == dimColResolvedFilterInfo.getEndIndexKey()) {
-      try {
-        FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
-            absoluteTableIdentifier, endKeys, segmentProperties);
-        FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
-            setOfEndKeyByteArray);
-      } catch (QueryExecutionException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-    }
-  }
-
-  /**
-   * Method will return the executer type for particular conditional resolver
-   * basically two types of executers will be formed for the conditional query.
-   *
-   * @return the filter executer type
-   */
-  @Override public FilterExecuterType getFilterExecuterType() {
-    switch (exp.getFilterExpressionType()) {
-      case NOT_EQUALS:
-      case NOT_IN:
-        return FilterExecuterType.EXCLUDE;
-
-      default:
-        return FilterExecuterType.INCLUDE;
-    }
-
-  }
-
-  @Override public Expression getFilterExpression() {
-    // TODO Auto-generated method stub
-    return exp;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java b/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
deleted file mode 100644
index a8f1df5..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
+++ /dev/null
@@ -1,98 +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.carbondata.query.filter.resolver;
-
-import java.io.Serializable;
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public interface FilterResolverIntf extends Serializable {
-
-  /**
-   * This API will resolve the filter expression and generates the
-   * dictionaries for executing/evaluating the filter expressions in the
-   * executer layer.
-   *
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) throws FilterUnsupportedException;
-
-  /**
-   * This API will provide the left column filter expression
-   * inorder to resolve the left expression filter.
-   *
-   * @return FilterResolverIntf
-   */
-  FilterResolverIntf getLeft();
-
-  /**
-   * API will provide the right column filter expression inorder to resolve
-   * the right expression filter.
-   *
-   * @return FilterResolverIntf
-   */
-  FilterResolverIntf getRight();
-
-  /**
-   * API will return the resolved filter instance, this instance will provide
-   * the resolved surrogates based on the applied filter
-   *
-   * @return DimColumnResolvedFilterInfo object
-   */
-  DimColumnResolvedFilterInfo getDimColResolvedFilterInfo();
-
-  /**
-   * API will get the start key based on the filter applied based on the key generator
-   *
-   * @param segmentProperties
-   * @param startKey
-   * @param setOfStartKeyByteArray
-   */
-  void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray);
-
-  /**
-   * API will read the end key based on the max surrogate of
-   * particular dimension column
-   *
-   * @param setOfEndKeyByteArray
-   * @param endKeys
-   * @return
-   */
-  void getEndKey(SegmentProperties segmentProperties, AbsoluteTableIdentifier tableIdentifier,
-      long[] endKeys, SortedMap<Integer, byte[]> setOfEndKeyByteArray);
-
-  /**
-   * API will return the filter executer type which will be used to evaluate
-   * the resolved filter while query execution
-   *
-   * @return FilterExecuterType.
-   */
-  FilterExecuterType getFilterExecuterType();
-
-  Expression getFilterExpression();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
deleted file mode 100644
index 8bf1395..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
+++ /dev/null
@@ -1,109 +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.carbondata.query.filter.resolver;
-
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public class LogicalFilterResolverImpl implements FilterResolverIntf {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 5734382980564402914L;
-
-  protected FilterResolverIntf leftEvalutor;
-
-  protected FilterResolverIntf rightEvalutor;
-
-  protected ExpressionType filterExpressionType;
-
-  public LogicalFilterResolverImpl(FilterResolverIntf leftEvalutor,
-      FilterResolverIntf rightEvalutor, ExpressionType filterExpressionType) {
-    this.leftEvalutor = leftEvalutor;
-    this.rightEvalutor = rightEvalutor;
-    this.filterExpressionType = filterExpressionType;
-  }
-
-  /**
-   * Logical filter resolver will return the left and right filter expresison
-   * node for filter evaluation, so in this instance no implementation is required.
-   *
-   * @param absoluteTableIdentifier
-   */
-  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
-
-  }
-
-  /**
-   * Since its a binary condition expresion the getLeft method will get the left
-   * node of filter expression
-   *
-   * @return FilterResolverIntf.
-   */
-  public FilterResolverIntf getLeft() {
-    return leftEvalutor;
-  }
-
-  /**
-   * Since its a binary condition expresion the getRight method will get the left
-   * node of filter expression
-   *
-   * @return FilterResolverIntf.
-   */
-  public FilterResolverIntf getRight() {
-    return rightEvalutor;
-  }
-
-  @Override public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
-    return null;
-  }
-
-  @Override public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
-
-  }
-
-  @Override public void getEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
-  }
-
-  @Override public FilterExecuterType getFilterExecuterType() {
-    switch (filterExpressionType) {
-      case OR:
-        return FilterExecuterType.OR;
-      case AND:
-        return FilterExecuterType.AND;
-
-      default:
-        return null;
-    }
-  }
-
-  @Override public Expression getFilterExpression() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java
deleted file mode 100644
index 516de9c..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/RestructureFilterResolverImpl.java
+++ /dev/null
@@ -1,210 +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.carbondata.query.filter.resolver;
-
-import java.util.List;
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-public class RestructureFilterResolverImpl implements FilterResolverIntf {
-  /**
-   *
-   */
-  private static final long serialVersionUID = -5399656036192814524L;
-
-  protected DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo;
-
-  private Expression exp;
-
-  private String defaultValue;
-
-  private int surrogate;
-
-  private boolean isExpressionResolve;
-
-  private boolean isIncludeFilter;
-
-  public RestructureFilterResolverImpl(Expression exp, String defaultValue, int surrogate,
-      boolean isExpressionResolve, boolean isIncludeFilter) {
-    dimColumnResolvedFilterInfo = new DimColumnResolvedFilterInfo();
-    this.exp = exp;
-    this.defaultValue = defaultValue;
-    this.surrogate = surrogate;
-    this.isExpressionResolve = isExpressionResolve;
-    this.isIncludeFilter = isIncludeFilter;
-  }
-
-  /**
-   * Method will resolve the filters and it will replace the newly added dimension with default
-   * value
-   *
-   * @param absoluteTableIdentifier
-   */
-  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
-
-    DimColumnResolvedFilterInfo dimColumnResolvedFilterInfo = new DimColumnResolvedFilterInfo();
-    if (!this.isExpressionResolve && exp instanceof BinaryConditionalExpression) {
-      BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression) exp;
-      Expression left = binaryConditionalExpression.getLeft();
-      Expression right = binaryConditionalExpression.getRight();
-      if (left instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) left;
-        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
-          isExpressionResolve = true;
-        } else {
-          // If imei=imei comes in filter condition then we need to
-          // skip processing of right expression.
-          // This flow has reached here assuming that this is a single
-          // column expression.
-          // we need to check if the other expression contains column
-          // expression or not in depth.
-          if (FilterUtil.checkIfExpressionContainsColumn(right)) {
-            isExpressionResolve = true;
-          } else {
-            dimColumnResolvedFilterInfo
-                .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
-            dimColumnResolvedFilterInfo.setFilterValues(
-                FilterUtil.getFilterListForRS(right, columnExpression, defaultValue, surrogate));
-          }
-        }
-      } else if (right instanceof ColumnExpression) {
-        ColumnExpression columnExpression = (ColumnExpression) right;
-        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
-          isExpressionResolve = true;
-        } else {
-
-          // If imei=imei comes in filter condition then we need to
-          // skip processing of right expression.
-          // This flow has reached here assuming that this is a single
-          // column expression.
-          // we need to check if the other expression contains column
-          // expression or not in depth.
-          if (checkIfExpressionContainsColumn(left)) {
-            isExpressionResolve = true;
-          } else {
-            dimColumnResolvedFilterInfo
-                .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
-            dimColumnResolvedFilterInfo.setFilterValues(
-                FilterUtil.getFilterListForRS(left, columnExpression, defaultValue, surrogate));
-          }
-        }
-      }
-    }
-    if (this.isExpressionResolve && exp instanceof ConditionalExpression) {
-      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
-      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
-      dimColumnResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
-      dimColumnResolvedFilterInfo.setFilterValues(FilterUtil
-          .getFilterListForAllMembersRS(exp, columnList.get(0), defaultValue, surrogate,
-              isIncludeFilter));
-    }
-
-  }
-
-  /**
-   * This method will check if a given expression contains a column expression recursively.
-   *
-   * @return boolean
-   */
-  private boolean checkIfExpressionContainsColumn(Expression expression) {
-    if (expression instanceof ColumnExpression) {
-      return true;
-    }
-    for (Expression child : expression.getChildren()) {
-      if (checkIfExpressionContainsColumn(child)) {
-        return true;
-      }
-    }
-
-    return false;
-  }
-
-  @Override public FilterResolverIntf getLeft() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override public FilterResolverIntf getRight() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  /**
-   * Method will return the DimColumnResolvedFilterInfo instance which consists
-   * the mapping of the respective dimension and its surrogates involved in
-   * filter expression.
-   *
-   * @return DimColumnResolvedFilterInfo
-   */
-  public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
-    return dimColumnResolvedFilterInfo;
-  }
-
-  /**
-   * For restructure resolver no implementation is required for getting
-   * the start key since it already has default values
-   */
-  @Override public void getStartKey(SegmentProperties segmentProperties, long[] startKeys,
-      SortedMap<Integer, byte[]> noDicStartKeys) {
-
-  }
-
-  /**
-   * For restructure resolver no implementation is required for getting
-   * the end  key since it already has default values
-   *
-   * @return IndexKey.
-   */
-  @Override public void getEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> noDicEndKeys) {
-  }
-
-  /**
-   * Method will get the executer type inorder to create filter executer tree
-   *
-   * @return FilterExecuterType
-   */
-  @Override public FilterExecuterType getFilterExecuterType() {
-    return FilterExecuterType.RESTRUCTURE;
-  }
-
-  @Override public Expression getFilterExpression() {
-    // TODO Auto-generated method stub
-    return exp;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.java
deleted file mode 100644
index ceccb12..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelFilterResolverImpl.java
+++ /dev/null
@@ -1,143 +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.carbondata.query.filter.resolver;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-
-public class RowLevelFilterResolverImpl extends ConditionalFilterResolverImpl {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 176122729713729929L;
-  protected boolean isExpressionResolve;
-  protected boolean isIncludeFilter;
-
-  private List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
-  private List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
-  private AbsoluteTableIdentifier tableIdentifier;
-
-  public RowLevelFilterResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
-    super(exp, isExpressionResolve, isIncludeFilter);
-    dimColEvaluatorInfoList =
-        new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(
-        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    this.tableIdentifier = tableIdentifier;
-  }
-
-  /**
-   * Method which will resolve the filter expression by converting the filter member
-   * to its assigned dictionary values.
-   */
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = null;
-    MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = null;
-    int index = 0;
-    if (exp instanceof ConditionalExpression) {
-      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
-      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
-      for (ColumnExpression columnExpression : columnList) {
-        if (columnExpression.isDimension()) {
-          dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
-          dimColumnEvaluatorInfo.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-          dimColumnEvaluatorInfo.setRowIndex(index++);
-          dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
-          dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
-          dimColEvaluatorInfoList.add(dimColumnEvaluatorInfo);
-        } else {
-          msrColumnEvalutorInfo = new MeasureColumnResolvedFilterInfo();
-          msrColumnEvalutorInfo.setRowIndex(index++);
-          msrColumnEvalutorInfo.setAggregator(
-              ((CarbonMeasure) columnExpression.getCarbonColumn()).getAggregateFunction());
-          msrColumnEvalutorInfo
-              .setColumnIndex(((CarbonMeasure) columnExpression.getCarbonColumn()).getOrdinal());
-          msrColumnEvalutorInfo.setType(columnExpression.getCarbonColumn().getDataType());
-          msrColEvalutorInfoList.add(msrColumnEvalutorInfo);
-        }
-      }
-    }
-  }
-
-  /**
-   * This method will provide the executer type to the callee inorder to identify
-   * the executer type for the filter resolution, Row level filter executer is a
-   * special executer since it get all the rows of the specified filter dimension
-   * and will be send to the spark for processing
-   */
-  @Override public FilterExecuterType getFilterExecuterType() {
-    return FilterExecuterType.ROWLEVEL;
-  }
-
-  /**
-   * Method will the read filter expression corresponding to the resolver.
-   * This method is required in row level executer inorder to evaluate the filter
-   * expression against spark, as mentioned above row level is a special type
-   * filter resolver.
-   *
-   * @return Expression
-   */
-  public Expression getFilterExpresion() {
-    return exp;
-  }
-
-  /**
-   * Method will return the DimColumnResolvedFilterInfo instance which consists
-   * the mapping of the respective dimension and its surrogates involved in
-   * filter expression.
-   *
-   * @return DimColumnResolvedFilterInfo
-   */
-  public List<DimColumnResolvedFilterInfo> getDimColEvaluatorInfoList() {
-    return dimColEvaluatorInfoList;
-  }
-
-  /**
-   * Method will return the DimColumnResolvedFilterInfo instance which containts
-   * measure level details.
-   *
-   * @return MeasureColumnResolvedFilterInfo
-   */
-  public List<MeasureColumnResolvedFilterInfo> getMsrColEvalutorInfoList() {
-    return msrColEvalutorInfoList;
-  }
-
-  /**
-   * Method will return table information which will be required for retrieving
-   * dictionary cache inorder to read all the members of respective dimension.
-   *
-   * @return AbsoluteTableIdentifier
-   */
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
deleted file mode 100644
index 6e220b7..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ /dev/null
@@ -1,232 +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.carbondata.query.filter.resolver;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.SortedMap;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
-import org.carbondata.query.expression.logical.BinaryLogicalExpression;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverImpl {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 6629319265336666789L;
-
-  private List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
-  private List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
-  private AbsoluteTableIdentifier tableIdentifier;
-
-  public RowLevelRangeFilterResolverImpl(Expression exp, boolean isExpressionResolve,
-      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
-    super(exp, isExpressionResolve, isIncludeFilter);
-    dimColEvaluatorInfoList =
-        new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(
-        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    this.tableIdentifier = tableIdentifier;
-  }
-
-  /**
-   * This method will return the filter values which is present in the range leve
-   * conditional expressions.
-   *
-   * @return
-   */
-  public byte[][] getFilterRangeValues() {
-    List<byte[]> filterValuesList = new ArrayList<byte[]>();
-    if (null != dimColEvaluatorInfoList.get(0).getFilterValues()) {
-      filterValuesList =
-          dimColEvaluatorInfoList.get(0).getFilterValues().getNoDictionaryFilterValuesList();
-      return filterValuesList.toArray((new byte[filterValuesList.size()][]));
-    }
-    return filterValuesList.toArray((new byte[filterValuesList.size()][]));
-
-  }
-
-  /**
-   * method will get the start key based on the filter surrogates
-   *
-   * @return start IndexKey
-   */
-  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
-      SortedMap<Integer, byte[]> noDictStartKeys) {
-    if (null == dimColEvaluatorInfoList.get(0).getStarIndexKey()) {
-      FilterUtil
-          .getStartKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
-              noDictStartKeys);
-    }
-  }
-
-  /**
-   * method will get the start key based on the filter surrogates
-   *
-   * @return end IndexKey
-   */
-  @Override public void getEndKey(SegmentProperties segmentProperties,
-      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> noDicEndKeys) {
-    if (null == dimColEvaluatorInfoList.get(0).getEndIndexKey()) {
-      try {
-        FilterUtil.getEndKey(dimColEvaluatorInfoList.get(0).getDimensionResolvedFilterInstance(),
-            absoluteTableIdentifier, endKeys, segmentProperties);
-        FilterUtil
-            .getEndKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
-                noDicEndKeys);
-      } catch (QueryExecutionException e) {
-        // TODO Auto-generated catch block
-        e.printStackTrace();
-      }
-    }
-  }
-
-  private List<byte[]> getNoDictionaryRangeValues() {
-    List<ExpressionResult> listOfExpressionResults = new ArrayList<ExpressionResult>(20);
-    if (this.getFilterExpression() instanceof BinaryConditionalExpression) {
-      listOfExpressionResults =
-          ((BinaryConditionalExpression) this.getFilterExpression()).getLiterals();
-    }
-    List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
-    for (ExpressionResult result : listOfExpressionResults) {
-      if (result.getString() == null) {
-        filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL.getBytes());
-        continue;
-      }
-      filterValuesList.add(result.getString().getBytes());
-    }
-    Comparator<byte[]> filterNoDictValueComaparator = new Comparator<byte[]>() {
-      @Override public int compare(byte[] filterMember1, byte[] filterMember2) {
-        return ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterMember1, filterMember2);
-      }
-
-    };
-    Collections.sort(filterValuesList, filterNoDictValueComaparator);
-    return filterValuesList;
-  }
-
-  /**
-   * Method which will resolve the filter expression by converting the filter
-   * member to its assigned dictionary values.
-   */
-  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = null;
-    MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = null;
-    int index = 0;
-    if (exp instanceof BinaryLogicalExpression) {
-      BinaryLogicalExpression conditionalExpression = (BinaryLogicalExpression) exp;
-      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
-      for (ColumnExpression columnExpression : columnList) {
-        if (columnExpression.isDimension()) {
-          dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
-          DimColumnFilterInfo filterInfo = new DimColumnFilterInfo();
-          dimColumnEvaluatorInfo.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-          //dimColumnEvaluatorInfo.se
-          dimColumnEvaluatorInfo.setRowIndex(index++);
-          dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
-          dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
-          filterInfo.setFilterListForNoDictionaryCols(getNoDictionaryRangeValues());
-          filterInfo.setIncludeFilter(isIncludeFilter);
-          dimColumnEvaluatorInfo.setFilterValues(filterInfo);
-          dimColumnEvaluatorInfo
-              .addDimensionResolvedFilterInstance(columnExpression.getDimension(), filterInfo);
-          dimColEvaluatorInfoList.add(dimColumnEvaluatorInfo);
-        } else {
-          msrColumnEvalutorInfo = new MeasureColumnResolvedFilterInfo();
-          msrColumnEvalutorInfo.setRowIndex(index++);
-          msrColumnEvalutorInfo.setAggregator(
-              ((CarbonMeasure) columnExpression.getCarbonColumn()).getAggregateFunction());
-          msrColumnEvalutorInfo
-              .setColumnIndex(((CarbonMeasure) columnExpression.getCarbonColumn()).getOrdinal());
-          msrColumnEvalutorInfo.setType(columnExpression.getCarbonColumn().getDataType());
-          msrColEvalutorInfoList.add(msrColumnEvalutorInfo);
-        }
-      }
-    }
-  }
-
-  /**
-   * Method will return the DimColumnResolvedFilterInfo instance which consists
-   * the mapping of the respective dimension and its surrogates involved in
-   * filter expression.
-   *
-   * @return DimColumnResolvedFilterInfo
-   */
-  public List<DimColumnResolvedFilterInfo> getDimColEvaluatorInfoList() {
-    return dimColEvaluatorInfoList;
-  }
-
-  /**
-   * Method will return the DimColumnResolvedFilterInfo instance which containts
-   * measure level details.
-   *
-   * @return MeasureColumnResolvedFilterInfo
-   */
-  public List<MeasureColumnResolvedFilterInfo> getMsrColEvalutorInfoList() {
-    return msrColEvalutorInfoList;
-  }
-
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
-  public Expression getFilterExpression() {
-    return this.exp;
-  }
-
-  /**
-   * This method will provide the executer type to the callee inorder to identify
-   * the executer type for the filter resolution, Row level filter executer is a
-   * special executer since it get all the rows of the specified filter dimension
-   * and will be send to the spark for processing
-   */
-  public FilterExecuterType getFilterExecuterType() {
-    switch (exp.getFilterExpressionType()) {
-      case GREATERTHAN:
-        return FilterExecuterType.ROWLEVEL_GREATERTHAN;
-      case GREATERTHAN_EQUALTO:
-        return FilterExecuterType.ROWLEVEL_GREATERTHAN_EQUALTO;
-      case LESSTHAN:
-        return FilterExecuterType.ROWLEVEL_LESSTHAN;
-      case LESSTHAN_EQUALTO:
-        return FilterExecuterType.ROWLEVEL_LESSTHAN_EQUALTO;
-
-      default:
-        return FilterExecuterType.ROWLEVEL;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/metadata/FilterResolverMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/metadata/FilterResolverMetadata.java b/core/src/main/java/org/carbondata/query/filter/resolver/metadata/FilterResolverMetadata.java
deleted file mode 100644
index 3f813de..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/metadata/FilterResolverMetadata.java
+++ /dev/null
@@ -1,62 +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.carbondata.query.filter.resolver.metadata;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-
-public class FilterResolverMetadata {
-  private AbsoluteTableIdentifier tableIdentifier;
-  private Expression expression;
-  private ColumnExpression columnExpression;
-  private boolean isIncludeFilter;
-
-  public AbsoluteTableIdentifier getTableIdentifier() {
-    return tableIdentifier;
-  }
-
-  public void setTableIdentifier(AbsoluteTableIdentifier tableIdentifier) {
-    this.tableIdentifier = tableIdentifier;
-  }
-
-  public Expression getExpression() {
-    return expression;
-  }
-
-  public void setExpression(Expression expression) {
-    this.expression = expression;
-  }
-
-  public ColumnExpression getColumnExpression() {
-    return columnExpression;
-  }
-
-  public void setColumnExpression(ColumnExpression columnExpression) {
-    this.columnExpression = columnExpression;
-  }
-
-  public boolean isIncludeFilter() {
-    return isIncludeFilter;
-  }
-
-  public void setIncludeFilter(boolean isIncludeFilter) {
-    this.isIncludeFilter = isIncludeFilter;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
deleted file mode 100644
index 4d1f827..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
+++ /dev/null
@@ -1,206 +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.carbondata.query.filter.resolver.resolverinfo;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.core.carbon.datastore.IndexKey;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.query.complex.querytypes.GenericQueryType;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.visitable.ResolvedFilterInfoVisitable;
-import org.carbondata.query.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-public class DimColumnResolvedFilterInfo implements Serializable, ResolvedFilterInfoVisitable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 3428115141211084114L;
-
-  /**
-   * column index in file
-   */
-  private int columnIndex = -1;
-
-  /**
-   * need compressed data from file
-   */
-  private boolean needCompressedData;
-
-  /**
-   * rowIndex
-   */
-  private int rowIndex = -1;
-
-  private boolean isDimensionExistsInCurrentSilce = true;
-
-  private int rsSurrogates;
-
-  private String defaultValue;
-
-  private transient Map<Integer, GenericQueryType> complexTypesWithBlockStartIndex;
-
-  private CarbonDimension dimension;
-
-  /**
-   * start index key of the block based on the keygenerator
-   */
-  private transient IndexKey starIndexKey;
-
-  /**
-   * end index key  which is been formed considering the max surrogate values
-   * from dictionary cache
-   */
-  private transient IndexKey endIndexKey;
-
-  /**
-   * reolved filter object of a particlar filter Expression.
-   */
-  private DimColumnFilterInfo resolvedFilterValueObj;
-
-  private Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionResolvedFilter;
-
-  public DimColumnResolvedFilterInfo() {
-    dimensionResolvedFilter = new HashMap<CarbonDimension, List<DimColumnFilterInfo>>(20);
-  }
-
-  public IndexKey getStarIndexKey() {
-    return starIndexKey;
-  }
-
-  public void setStarIndexKey(IndexKey starIndexKey) {
-    this.starIndexKey = starIndexKey;
-  }
-
-  public IndexKey getEndIndexKey() {
-    return endIndexKey;
-  }
-
-  public void setEndIndexKey(IndexKey endIndexKey) {
-    this.endIndexKey = endIndexKey;
-  }
-
-  public void addDimensionResolvedFilterInstance(CarbonDimension dimension,
-      DimColumnFilterInfo filterResolvedObj) {
-    List<DimColumnFilterInfo> currentVals = dimensionResolvedFilter.get(dimension);
-    if (null == currentVals) {
-      currentVals = new ArrayList<DimColumnFilterInfo>(20);
-      currentVals.add(filterResolvedObj);
-      dimensionResolvedFilter.put(dimension, currentVals);
-    } else {
-      currentVals.add(filterResolvedObj);
-    }
-  }
-
-  public Map<CarbonDimension, List<DimColumnFilterInfo>> getDimensionResolvedFilterInstance() {
-    return dimensionResolvedFilter;
-  }
-
-  public Map<Integer, GenericQueryType> getComplexTypesWithBlockStartIndex() {
-    return complexTypesWithBlockStartIndex;
-  }
-
-  public void setComplexTypesWithBlockStartIndex(
-      Map<Integer, GenericQueryType> complexTypesWithBlockStartIndex) {
-    this.complexTypesWithBlockStartIndex = complexTypesWithBlockStartIndex;
-  }
-
-  public CarbonDimension getDimension() {
-    return dimension;
-  }
-
-  public void setDimension(CarbonDimension dimension) {
-    this.dimension = dimension;
-  }
-
-  public int getColumnIndex() {
-    return columnIndex;
-  }
-
-  public void setColumnIndex(int columnIndex) {
-    this.columnIndex = columnIndex;
-  }
-
-  public boolean isNeedCompressedData() {
-    return needCompressedData;
-  }
-
-  public void setNeedCompressedData(boolean needCompressedData) {
-    this.needCompressedData = needCompressedData;
-  }
-
-  public DimColumnFilterInfo getFilterValues() {
-    return resolvedFilterValueObj;
-  }
-
-  public void setFilterValues(final DimColumnFilterInfo resolvedFilterValueObj) {
-    this.resolvedFilterValueObj = resolvedFilterValueObj;
-  }
-
-  public int getRowIndex() {
-    return rowIndex;
-  }
-
-  public void setRowIndex(int rowIndex) {
-    this.rowIndex = rowIndex;
-  }
-
-  public boolean isDimensionExistsInCurrentSilce() {
-    return isDimensionExistsInCurrentSilce;
-  }
-
-  public void setDimensionExistsInCurrentSilce(boolean isDimensionExistsInCurrentSilce) {
-    this.isDimensionExistsInCurrentSilce = isDimensionExistsInCurrentSilce;
-  }
-
-  public int getRsSurrogates() {
-    return rsSurrogates;
-  }
-
-  public void setRsSurrogates(int rsSurrogates) {
-    this.rsSurrogates = rsSurrogates;
-  }
-
-  public String getDefaultValue() {
-    return defaultValue;
-  }
-
-  public void setDefaultValue(String defaultValue) {
-    this.defaultValue = defaultValue;
-  }
-
-  @Override public void populateFilterInfoBasedOnColumnType(ResolvedFilterInfoVisitorIntf visitor,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException {
-    if (null != visitor) {
-      visitor.populateFilterResolvedInfo(this, metadata);
-      this.addDimensionResolvedFilterInstance(metadata.getColumnExpression().getDimension(),
-          this.getFilterValues());
-      this.setDimension(metadata.getColumnExpression().getDimension());
-      this.setColumnIndex(metadata.getColumnExpression().getDimension().getOrdinal());
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
deleted file mode 100644
index e6877d5..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
+++ /dev/null
@@ -1,105 +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.carbondata.query.filter.resolver.resolverinfo;
-
-import java.io.Serializable;
-
-public class MeasureColumnResolvedFilterInfo implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 4222568289115151561L;
-
-  private int columnIndex = -1;
-
-  private int rowIndex = -1;
-
-  private Object uniqueValue;
-
-  private String aggregator;
-
-  private boolean isMeasureExistsInCurrentSlice = true;
-
-  private Object defaultValue;
-
-  private org.carbondata.core.carbon.metadata.datatype.DataType type;
-
-  public int getColumnIndex() {
-    return columnIndex;
-  }
-
-  public void setColumnIndex(int columnIndex) {
-    this.columnIndex = columnIndex;
-  }
-
-  public int getRowIndex() {
-    return rowIndex;
-  }
-
-  public void setRowIndex(int rowIndex) {
-    this.rowIndex = rowIndex;
-  }
-
-  public Object getUniqueValue() {
-    return uniqueValue;
-  }
-
-  public void setUniqueValue(Object uniqueValue) {
-    this.uniqueValue = uniqueValue;
-  }
-
-  public org.carbondata.core.carbon.metadata.datatype.DataType getType() {
-    return type;
-  }
-
-  public void setType(org.carbondata.core.carbon.metadata.datatype.DataType dataType) {
-    this.type = dataType;
-  }
-
-  /**
-   * @return Returns the aggregator.
-   */
-  public String getAggregator() {
-    return aggregator;
-  }
-
-  /**
-   * @param aggregator The aggregator to set.
-   */
-  public void setAggregator(String aggregator) {
-    this.aggregator = aggregator;
-  }
-
-  public boolean isMeasureExistsInCurrentSlice() {
-    return isMeasureExistsInCurrentSlice;
-  }
-
-  public void setMeasureExistsInCurrentSlice(boolean isMeasureExistsInCurrentSlice) {
-    this.isMeasureExistsInCurrentSlice = isMeasureExistsInCurrentSlice;
-  }
-
-  public Object getDefaultValue() {
-    return defaultValue;
-  }
-
-  public void setDefaultValue(double defaultValue) {
-    this.defaultValue = defaultValue;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
deleted file mode 100644
index 7f31cb4..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.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.carbondata.query.filter.resolver.resolverinfo.visitable;
-
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
-
-public interface ResolvedFilterInfoVisitable {
-  /**
-   * This visitable method will visit through the visitor classes which is passed as parameter
-   * and based on different visitor the visitable filter instance will be resolved.
-   *
-   * @param visitor
-   * @param metadata
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  void populateFilterInfoBasedOnColumnType(ResolvedFilterInfoVisitorIntf visitor,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException;
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
deleted file mode 100644
index 52d0e26..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
+++ /dev/null
@@ -1,80 +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.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-public class CustomTypeDictionaryVisitor implements ResolvedFilterInfoVisitorIntf {
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CustomTypeDictionaryVisitor.class.getName());
-
-  /**
-   * This Visitor method is been used to resolve or populate the filter details
-   * by using custom type dictionary value, the filter membrers will be resolved using
-   * custom type function which will generate dictionary for the direct column type filter members
-   *
-   * @param visitableObj
-   * @param metadata
-   * @throws FilterUnsupportedException
-   */
-  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException {
-    DimColumnFilterInfo resolvedFilterObject = null;
-
-    List<String> evaluateResultListFinal =
-        metadata.getExpression().evaluate(null).getListAsString();
-    resolvedFilterObject = getDirectDictionaryValKeyMemberForFilter(metadata.getTableIdentifier(),
-        metadata.getColumnExpression(), evaluateResultListFinal, metadata.isIncludeFilter());
-    visitableObj.setFilterValues(resolvedFilterObject);
-  }
-
-  private DimColumnFilterInfo getDirectDictionaryValKeyMemberForFilter(
-      AbsoluteTableIdentifier tableIdentifier, ColumnExpression columnExpression,
-      List<String> evaluateResultListFinal, boolean isIncludeFilter) {
-    List<Integer> surrogates = new ArrayList<Integer>(20);
-    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-        .getDirectDictionaryGenerator(columnExpression.getDimension().getDataType());
-    // Reading the dictionary value direct
-    for (String filterMember : evaluateResultListFinal) {
-      surrogates.add(directDictionaryGenerator.generateDirectSurrogateKey(filterMember));
-    }
-    Collections.sort(surrogates);
-    DimColumnFilterInfo columnFilterInfo = null;
-    if (surrogates.size() > 0) {
-      columnFilterInfo = new DimColumnFilterInfo();
-      columnFilterInfo.setIncludeFilter(isIncludeFilter);
-      columnFilterInfo.setFilterList(surrogates);
-    }
-    return columnFilterInfo;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
deleted file mode 100644
index ee3bd4c..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
+++ /dev/null
@@ -1,58 +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.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-public class DictionaryColumnVisitor implements ResolvedFilterInfoVisitorIntf {
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DictionaryColumnVisitor.class.getName());
-
-  /**
-   * This Visitor method is used to populate the visitableObj with direct dictionary filter details
-   * where the filters values will be resolve using dictionary cache.
-   *
-   * @param visitableObj
-   * @param metadata
-   * @throws QueryExecutionException
-   */
-  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException {
-    DimColumnFilterInfo resolvedFilterObject = null;
-    List<String> evaluateResultListFinal =
-        metadata.getExpression().evaluate(null).getListAsString();
-    try {
-      resolvedFilterObject = FilterUtil
-          .getFilterValues(metadata.getTableIdentifier(), metadata.getColumnExpression(),
-              evaluateResultListFinal, metadata.isIncludeFilter());
-    } catch (QueryExecutionException e) {
-      throw new FilterUnsupportedException(e);
-    }
-    visitableObj.setFilterValues(resolvedFilterObject);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
deleted file mode 100644
index 3a38569..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
+++ /dev/null
@@ -1,45 +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.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.query.expression.ColumnExpression;
-
-public class FilterInfoTypeVisitorFactory {
-
-  /**
-   * This factory method will be used in order to get the visitor instance based on the
-   * column expression metadata where filters has been applied.
-   *
-   * @param columnExpression
-   * @return
-   */
-  public static ResolvedFilterInfoVisitorIntf getResolvedFilterInfoVisitor(
-      ColumnExpression columnExpression) {
-    if (columnExpression.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-      return new CustomTypeDictionaryVisitor();
-    } else if (!columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)) {
-      return new NoDictionaryTypeVisitor();
-    } else if (columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)) {
-      return new DictionaryColumnVisitor();
-    }
-
-    return null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
deleted file mode 100644
index fbafa7d..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.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.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-import org.carbondata.query.schema.metadata.DimColumnFilterInfo;
-
-public class NoDictionaryTypeVisitor implements ResolvedFilterInfoVisitorIntf {
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(NoDictionaryTypeVisitor.class.getName());
-
-  /**
-   * Visitor Method will update the filter related details in visitableObj, For no dictionary
-   * type columns the filter members will resolved directly, no need to look up in dictionary
-   * since it will not be part of dictionary, directly the actual data can be converted as
-   * byte[] and can be set. this type of encoding is effective when the particular column
-   * is having very high cardinality.
-   *
-   * @param visitableObj
-   * @param metadata
-   * @throws FilterUnsupportedException
-   */
-  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException {
-    DimColumnFilterInfo resolvedFilterObject = null;
-    List<String> evaluateResultListFinal =
-        metadata.getExpression().evaluate(null).getListAsString();
-    resolvedFilterObject = FilterUtil
-        .getNoDictionaryValKeyMemberForFilter(metadata.getTableIdentifier(),
-            metadata.getColumnExpression(), evaluateResultListFinal, metadata.isIncludeFilter());
-    visitableObj.setFilterValues(resolvedFilterObject);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java b/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
deleted file mode 100644
index 44d942d..0000000
--- a/core/src/main/java/org/carbondata/query/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
+++ /dev/null
@@ -1,40 +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.carbondata.query.filter.resolver.resolverinfo.visitor;
-
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.metadata.FilterResolverMetadata;
-import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
-
-public interface ResolvedFilterInfoVisitorIntf {
-
-  /**
-   * Visitor pattern is been used in this scenario inorder to populate the
-   * dimColResolvedFilterInfo visitable object with filter member values based
-   * on the visitor type, currently there 3 types of visitors custom,direct
-   * and no dictionary, all types of visitor populate the visitable instance
-   * as per its buisness logic which is different for all the visitors.
-   *
-   * @param visitableObj
-   * @param metadata
-   * @throws QueryExecutionException
-   */
-  void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
-      FilterResolverMetadata metadata) throws FilterUnsupportedException;
-}


[32/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java b/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java
deleted file mode 100644
index d79ea5d..0000000
--- a/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java
+++ /dev/null
@@ -1,418 +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.carbondata.query.util;
-
-import java.io.ByteArrayInputStream;
-import java.io.IOException;
-import java.io.ObjectInputStream;
-import java.nio.ByteBuffer;
-import java.util.ArrayList;
-import java.util.BitSet;
-import java.util.Iterator;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
-import org.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
-import org.carbondata.core.carbon.metadata.blocklet.SegmentInfo;
-import org.carbondata.core.carbon.metadata.blocklet.compressor.ChunkCompressorMeta;
-import org.carbondata.core.carbon.metadata.blocklet.compressor.CompressionCodec;
-import org.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
-import org.carbondata.core.carbon.metadata.blocklet.datachunk.PresenceMeta;
-import org.carbondata.core.carbon.metadata.blocklet.index.BlockletBTreeIndex;
-import org.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
-import org.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
-import org.carbondata.core.carbon.metadata.blocklet.sort.SortState;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.metadata.ValueEncoderMeta;
-import org.carbondata.core.reader.CarbonFooterReader;
-import org.carbondata.core.util.ByteUtil;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.format.FileFooter;
-
-/**
- * Below class will be used to convert the thrift object of data file
- * meta data to wrapper object
- */
-public class DataFileFooterConverter {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataFileFooterConverter.class.getName());
-
-  /**
-   * Below method will be used to convert thrift file meta to wrapper file meta
-   */
-  public DataFileFooter readDataFileFooter(String filePath, long blockOffset, long blockLength)
-      throws IOException {
-    DataFileFooter dataFileFooter = new DataFileFooter();
-    FileHolder fileReader = null;
-    try {
-      long completeBlockLength = blockOffset + blockLength;
-      long footerPointer = completeBlockLength - 8;
-      fileReader = FileFactory.getFileHolder(FileFactory.getFileType(filePath));
-      long actualFooterOffset = fileReader.readLong(filePath, footerPointer);
-      CarbonFooterReader reader = new CarbonFooterReader(filePath, actualFooterOffset);
-      FileFooter footer = reader.readFooter();
-      dataFileFooter.setVersionId(footer.getVersion());
-      dataFileFooter.setNumberOfRows(footer.getNum_rows());
-      dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));
-      List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
-      List<org.carbondata.format.ColumnSchema> table_columns = footer.getTable_columns();
-      for (int i = 0; i < table_columns.size(); i++) {
-        columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
-      }
-      dataFileFooter.setColumnInTable(columnSchemaList);
-
-      List<org.carbondata.format.BlockletIndex> leaf_node_indices_Thrift =
-          footer.getBlocklet_index_list();
-      List<BlockletIndex> blockletIndexList = new ArrayList<BlockletIndex>();
-      for (int i = 0; i < leaf_node_indices_Thrift.size(); i++) {
-        BlockletIndex blockletIndex = getBlockletIndex(leaf_node_indices_Thrift.get(i));
-        blockletIndexList.add(blockletIndex);
-      }
-
-      List<org.carbondata.format.BlockletInfo> leaf_node_infos_Thrift =
-          footer.getBlocklet_info_list();
-      List<BlockletInfo> blockletInfoList = new ArrayList<BlockletInfo>();
-      for (int i = 0; i < leaf_node_infos_Thrift.size(); i++) {
-        BlockletInfo blockletInfo = getBlockletInfo(leaf_node_infos_Thrift.get(i));
-        blockletInfo.setBlockletIndex(blockletIndexList.get(i));
-        blockletInfoList.add(blockletInfo);
-      }
-      dataFileFooter.setBlockletList(blockletInfoList);
-      dataFileFooter.setBlockletIndex(getBlockletIndexForDataFileFooter(blockletIndexList));
-    } finally {
-      if (null != fileReader) {
-        fileReader.finish();
-      }
-    }
-    return dataFileFooter;
-  }
-
-  /**
-   * Below method will be used to get blocklet index for data file meta
-   *
-   * @param blockletIndexList
-   * @return blocklet index
-   */
-  private BlockletIndex getBlockletIndexForDataFileFooter(List<BlockletIndex> blockletIndexList) {
-    BlockletIndex blockletIndex = new BlockletIndex();
-    BlockletBTreeIndex blockletBTreeIndex = new BlockletBTreeIndex();
-    blockletBTreeIndex.setStartKey(blockletIndexList.get(0).getBtreeIndex().getStartKey());
-    blockletBTreeIndex
-        .setEndKey(blockletIndexList.get(blockletIndexList.size() - 1).getBtreeIndex().getEndKey());
-    blockletIndex.setBtreeIndex(blockletBTreeIndex);
-    byte[][] currentMinValue = blockletIndexList.get(0).getMinMaxIndex().getMinValues().clone();
-    byte[][] currentMaxValue = blockletIndexList.get(0).getMinMaxIndex().getMaxValues().clone();
-    byte[][] minValue = null;
-    byte[][] maxValue = null;
-    for (int i = 1; i < blockletIndexList.size(); i++) {
-      minValue = blockletIndexList.get(i).getMinMaxIndex().getMinValues();
-      maxValue = blockletIndexList.get(i).getMinMaxIndex().getMaxValues();
-      for (int j = 0; j < maxValue.length; j++) {
-        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMinValue[j], minValue[j]) > 0) {
-          currentMinValue[j] = minValue[j].clone();
-        }
-        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMaxValue[j], maxValue[j]) < 0) {
-          currentMaxValue[j] = maxValue[j].clone();
-        }
-      }
-    }
-
-    BlockletMinMaxIndex minMax = new BlockletMinMaxIndex();
-    minMax.setMaxValues(currentMaxValue);
-    minMax.setMinValues(currentMinValue);
-    blockletIndex.setMinMaxIndex(minMax);
-    return blockletIndex;
-  }
-
-  private ColumnSchema thriftColumnSchmeaToWrapperColumnSchema(
-      org.carbondata.format.ColumnSchema externalColumnSchema) {
-    ColumnSchema wrapperColumnSchema = new ColumnSchema();
-    wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
-    wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
-    wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
-    wrapperColumnSchema
-        .setDataType(thriftDataTyopeToWrapperDataType(externalColumnSchema.data_type));
-    wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
-    List<Encoding> encoders = new ArrayList<Encoding>();
-    for (org.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {
-      encoders.add(fromExternalToWrapperEncoding(encoder));
-    }
-    wrapperColumnSchema.setEncodingList(encoders);
-    wrapperColumnSchema.setNumberOfChild(externalColumnSchema.getNum_child());
-    wrapperColumnSchema.setPrecision(externalColumnSchema.getPrecision());
-    wrapperColumnSchema.setColumnGroup(externalColumnSchema.getColumn_group_id());
-    wrapperColumnSchema.setScale(externalColumnSchema.getScale());
-    wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
-    wrapperColumnSchema.setAggregateFunction(externalColumnSchema.getAggregate_function());
-    return wrapperColumnSchema;
-  }
-
-  /**
-   * Below method is to convert the blocklet info of the thrift to wrapper
-   * blocklet info
-   *
-   * @param blockletInfoThrift blocklet info of the thrift
-   * @return blocklet info wrapper
-   */
-  private BlockletInfo getBlockletInfo(org.carbondata.format.BlockletInfo blockletInfoThrift) {
-    BlockletInfo blockletInfo = new BlockletInfo();
-    List<DataChunk> dimensionColumnChunk = new ArrayList<DataChunk>();
-    List<DataChunk> measureChunk = new ArrayList<DataChunk>();
-    Iterator<org.carbondata.format.DataChunk> column_data_chunksIterator =
-        blockletInfoThrift.getColumn_data_chunksIterator();
-    while (column_data_chunksIterator.hasNext()) {
-      org.carbondata.format.DataChunk next = column_data_chunksIterator.next();
-      if (next.isRowMajor()) {
-        dimensionColumnChunk.add(getDataChunk(next, false));
-      } else if (next.getEncoders().contains(org.carbondata.format.Encoding.DELTA)) {
-        measureChunk.add(getDataChunk(next, true));
-      } else {
-
-        dimensionColumnChunk.add(getDataChunk(next, false));
-      }
-    }
-    blockletInfo.setDimensionColumnChunk(dimensionColumnChunk);
-    blockletInfo.setMeasureColumnChunk(measureChunk);
-    blockletInfo.setNumberOfRows(blockletInfoThrift.getNum_rows());
-    return blockletInfo;
-  }
-
-  /**
-   * Below method is convert the thrift encoding to wrapper encoding
-   *
-   * @param encoderThrift thrift encoding
-   * @return wrapper encoding
-   */
-  private Encoding fromExternalToWrapperEncoding(org.carbondata.format.Encoding encoderThrift) {
-    switch (encoderThrift) {
-      case DICTIONARY:
-        return Encoding.DICTIONARY;
-      case DELTA:
-        return Encoding.DELTA;
-      case RLE:
-        return Encoding.RLE;
-      case INVERTED_INDEX:
-        return Encoding.INVERTED_INDEX;
-      case BIT_PACKED:
-        return Encoding.BIT_PACKED;
-      case DIRECT_DICTIONARY:
-        return Encoding.DIRECT_DICTIONARY;
-      default:
-        return Encoding.DICTIONARY;
-    }
-  }
-
-  /**
-   * Below method will be used to convert the thrift compression to wrapper
-   * compression codec
-   *
-   * @param compressionCodecThrift
-   * @return wrapper compression codec
-   */
-  private CompressionCodec getCompressionCodec(
-      org.carbondata.format.CompressionCodec compressionCodecThrift) {
-    switch (compressionCodecThrift) {
-      case SNAPPY:
-        return CompressionCodec.SNAPPY;
-      default:
-        return CompressionCodec.SNAPPY;
-    }
-  }
-
-  /**
-   * Below method will be used to convert thrift segment object to wrapper
-   * segment object
-   *
-   * @param segmentInfo thrift segment info object
-   * @return wrapper segment info object
-   */
-  private SegmentInfo getSegmentInfo(org.carbondata.format.SegmentInfo segmentInfo) {
-    SegmentInfo info = new SegmentInfo();
-    int[] cardinality = new int[segmentInfo.getColumn_cardinalities().size()];
-    for (int i = 0; i < cardinality.length; i++) {
-      cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
-    }
-    info.setColumnCardinality(cardinality);
-    info.setNumberOfColumns(segmentInfo.getNum_cols());
-    return info;
-  }
-
-  /**
-   * Below method will be used to convert the blocklet index of thrift to
-   * wrapper
-   *
-   * @param blockletIndexThrift
-   * @return blocklet index wrapper
-   */
-  private BlockletIndex getBlockletIndex(org.carbondata.format.BlockletIndex blockletIndexThrift) {
-    org.carbondata.format.BlockletBTreeIndex btreeIndex = blockletIndexThrift.getB_tree_index();
-    org.carbondata.format.BlockletMinMaxIndex minMaxIndex = blockletIndexThrift.getMin_max_index();
-    return new BlockletIndex(
-        new BlockletBTreeIndex(btreeIndex.getStart_key(), btreeIndex.getEnd_key()),
-        new BlockletMinMaxIndex(minMaxIndex.getMin_values(), minMaxIndex.getMax_values()));
-  }
-
-  /**
-   * Below method will be used to convert the thrift compression meta to
-   * wrapper chunk compression meta
-   *
-   * @param chunkCompressionMetaThrift
-   * @return chunkCompressionMetaWrapper
-   */
-  private ChunkCompressorMeta getChunkCompressionMeta(
-      org.carbondata.format.ChunkCompressionMeta chunkCompressionMetaThrift) {
-    ChunkCompressorMeta compressorMeta = new ChunkCompressorMeta();
-    compressorMeta
-        .setCompressor(getCompressionCodec(chunkCompressionMetaThrift.getCompression_codec()));
-    compressorMeta.setCompressedSize(chunkCompressionMetaThrift.getTotal_compressed_size());
-    compressorMeta.setUncompressedSize(chunkCompressionMetaThrift.getTotal_uncompressed_size());
-    return compressorMeta;
-  }
-
-  /**
-   * Below method will be used to convert the thrift data type to wrapper data
-   * type
-   *
-   * @param dataTypeThrift
-   * @return dataType wrapper
-   */
-  private DataType thriftDataTyopeToWrapperDataType(org.carbondata.format.DataType dataTypeThrift) {
-    switch (dataTypeThrift) {
-      case STRING:
-        return DataType.STRING;
-      case INT:
-        return DataType.INT;
-      case LONG:
-        return DataType.LONG;
-      case DOUBLE:
-        return DataType.DOUBLE;
-      case DECIMAL:
-        return DataType.DECIMAL;
-      case TIMESTAMP:
-        return DataType.TIMESTAMP;
-      case ARRAY:
-        return DataType.ARRAY;
-      case STRUCT:
-        return DataType.STRUCT;
-      default:
-        return DataType.STRING;
-    }
-  }
-
-  /**
-   * Below method will be used to convert the thrift presence meta to wrapper
-   * presence meta
-   *
-   * @param presentMetadataThrift
-   * @return wrapper presence meta
-   */
-  private PresenceMeta getPresenceMeta(org.carbondata.format.PresenceMeta presentMetadataThrift) {
-    PresenceMeta presenceMeta = new PresenceMeta();
-    presenceMeta.setRepresentNullValues(presentMetadataThrift.isRepresents_presence());
-    presenceMeta.setBitSet(BitSet.valueOf(presentMetadataThrift.getPresent_bit_stream()));
-    return presenceMeta;
-  }
-
-  /**
-   * Below method will be used to convert the thrift object to wrapper object
-   *
-   * @param sortStateThrift
-   * @return wrapper sort state object
-   */
-  private SortState getSortState(org.carbondata.format.SortState sortStateThrift) {
-    if (sortStateThrift == org.carbondata.format.SortState.SORT_EXPLICIT) {
-      return SortState.SORT_EXPLICT;
-    } else if (sortStateThrift == org.carbondata.format.SortState.SORT_NATIVE) {
-      return SortState.SORT_NATIVE;
-    } else {
-      return SortState.SORT_NONE;
-    }
-  }
-
-  /**
-   * Below method will be used to convert the thrift data chunk to wrapper
-   * data chunk
-   *
-   * @param datachunkThrift
-   * @return wrapper data chunk
-   */
-  private DataChunk getDataChunk(org.carbondata.format.DataChunk datachunkThrift,
-      boolean isPresenceMetaPresent) {
-    DataChunk dataChunk = new DataChunk();
-    dataChunk.setColumnUniqueIdList(datachunkThrift.getColumn_ids());
-    dataChunk.setDataPageLength(datachunkThrift.getData_page_length());
-    dataChunk.setDataPageOffset(datachunkThrift.getData_page_offset());
-    if (isPresenceMetaPresent) {
-      dataChunk.setNullValueIndexForColumn(getPresenceMeta(datachunkThrift.getPresence()));
-    }
-    dataChunk.setRlePageLength(datachunkThrift.getRle_page_length());
-    dataChunk.setRlePageOffset(datachunkThrift.getRle_page_offset());
-    dataChunk.setRowMajor(datachunkThrift.isRowMajor());
-    dataChunk.setRowIdPageLength(datachunkThrift.getRowid_page_length());
-    dataChunk.setRowIdPageOffset(datachunkThrift.getRowid_page_offset());
-    dataChunk.setSortState(getSortState(datachunkThrift.getSort_state()));
-    dataChunk.setChunkCompressionMeta(getChunkCompressionMeta(datachunkThrift.getChunk_meta()));
-    List<Encoding> encodingList = new ArrayList<Encoding>(datachunkThrift.getEncoders().size());
-    for (int i = 0; i < datachunkThrift.getEncoders().size(); i++) {
-      encodingList.add(fromExternalToWrapperEncoding(datachunkThrift.getEncoders().get(i)));
-    }
-    dataChunk.setEncoderList(encodingList);
-    if (encodingList.contains(Encoding.DELTA)) {
-      List<ByteBuffer> thriftEncoderMeta = datachunkThrift.getEncoder_meta();
-      List<ValueEncoderMeta> encodeMetaList =
-          new ArrayList<ValueEncoderMeta>(thriftEncoderMeta.size());
-      for (int i = 0; i < thriftEncoderMeta.size(); i++) {
-        encodeMetaList.add(deserializeEncoderMeta(thriftEncoderMeta.get(i).array()));
-      }
-      dataChunk.setValueEncoderMeta(encodeMetaList);
-    }
-    return dataChunk;
-  }
-
-  /**
-   * Below method will be used to convert the encode metadata to
-   * ValueEncoderMeta object
-   *
-   * @param encoderMeta
-   * @return ValueEncoderMeta object
-   */
-  private ValueEncoderMeta deserializeEncoderMeta(byte[] encoderMeta) {
-    // TODO : should remove the unnecessary fields.
-    ByteArrayInputStream aos = null;
-    ObjectInputStream objStream = null;
-    ValueEncoderMeta meta = null;
-    try {
-      aos = new ByteArrayInputStream(encoderMeta);
-      objStream = new ObjectInputStream(aos);
-      meta = (ValueEncoderMeta) objStream.readObject();
-    } catch (ClassNotFoundException e) {
-      LOGGER.error(e);
-    } catch (IOException e) {
-      CarbonUtil.closeStreams(objStream);
-    }
-    return meta;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java b/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java
new file mode 100644
index 0000000..f214a9d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/collector/ScannedResultCollector.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.collector;
+
+import org.carbondata.scan.result.AbstractScannedResult;
+import org.carbondata.scan.result.Result;
+
+/**
+ * Interface which will be used to aggregate the scan result
+ */
+public interface ScannedResultCollector {
+
+  /**
+   * Below method will be used to aggregate the scanned result
+   *
+   * @param scannedResult scanned result
+   * @return how many records was aggregated
+   */
+  int collectData(AbstractScannedResult scannedResult, int batchSize);
+
+  /**
+   * Below method will be used to get the aggregated result
+   *
+   * @return
+   */
+  Result getCollectedResult();
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java b/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java
new file mode 100644
index 0000000..92507ea
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/collector/impl/ListBasedResultCollector.java
@@ -0,0 +1,212 @@
+/*
+ * 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.carbondata.scan.collector.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.scan.collector.ScannedResultCollector;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.util.QueryUtil;
+import org.carbondata.scan.result.AbstractScannedResult;
+import org.carbondata.scan.result.ListBasedResultWrapper;
+import org.carbondata.scan.result.Result;
+import org.carbondata.scan.result.impl.ListBasedResult;
+import org.carbondata.scan.util.DataTypeUtil;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
+
+/**
+ * It is not a collector it is just a scanned result holder.
+ *
+ */
+public class ListBasedResultCollector implements ScannedResultCollector {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ListBasedResultCollector.class.getName());
+
+  /**
+   * to keep a track of number of row processed to handle limit push down in
+   * case of detail query scenario
+   */
+  private int rowCounter;
+
+  /**
+   * dimension values list
+   */
+  private List<ListBasedResultWrapper> listBasedResult;
+
+  /**
+   * restructuring info
+   */
+  private KeyStructureInfo restructureInfos;
+
+  /**
+   * table block execution infos
+   */
+  private BlockExecutionInfo tableBlockExecutionInfos;
+
+  private int[] measuresOrdinal;
+
+  /**
+   * to check whether measure exists in current table block or not this to
+   * handle restructuring scenario
+   */
+  private boolean[] isMeasureExistsInCurrentBlock;
+
+  /**
+   * default value of the measures in case of restructuring some measure wont
+   * be present in the table so in that default value will be used to
+   * aggregate the data for that measure columns
+   */
+  private Object[] measureDefaultValue;
+
+  /**
+   * measure datatypes.
+   */
+  private DataType[] measureDatatypes;
+
+  public ListBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
+    this.tableBlockExecutionInfos = blockExecutionInfos;
+    restructureInfos = blockExecutionInfos.getKeyStructureInfo();
+    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
+    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
+    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
+    this.measureDatatypes = tableBlockExecutionInfos.getAggregatorInfo().getMeasureDataTypes();
+  }
+
+  @Override
+  /**
+   * This method will add a record both key and value to list object
+   * it will keep track of how many record is processed, to handle limit scenario
+   * @param scanned result
+   *
+   */
+  public int collectData(AbstractScannedResult scannedResult, int batchSize) {
+    this.listBasedResult =
+        new ArrayList<>(batchSize);
+    boolean isMsrsPresent = measureDatatypes.length > 0;
+    ByteArrayWrapper wrapper = null;
+    // scan the record and add to list
+    ListBasedResultWrapper resultWrapper;
+    int rowCounter = 0;
+    while (scannedResult.hasNext() && rowCounter < batchSize) {
+      resultWrapper = new ListBasedResultWrapper();
+      if(tableBlockExecutionInfos.isDimensionsExistInQuery()) {
+        wrapper = new ByteArrayWrapper();
+        wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
+        wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
+        wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
+        resultWrapper.setKey(wrapper);
+      } else {
+        scannedResult.incrementCounter();
+      }
+      if(isMsrsPresent) {
+        Object[] msrValues = new Object[measureDatatypes.length];
+        fillMeasureData(msrValues, scannedResult);
+        resultWrapper.setValue(msrValues);
+      }
+      listBasedResult.add(resultWrapper);
+      rowCounter++;
+    }
+    return rowCounter;
+  }
+
+  private void fillMeasureData(Object[] msrValues, AbstractScannedResult scannedResult) {
+    for (short i = 0; i < measuresOrdinal.length; i++) {
+      // if measure exists is block then pass measure column
+      // data chunk to the collector
+      if (isMeasureExistsInCurrentBlock[i]) {
+        msrValues[i] =
+            getMeasureData(scannedResult.getMeasureChunk(measuresOrdinal[i]),
+                scannedResult.getCurrenrRowId(),measureDatatypes[i]);
+      } else {
+        // if not then get the default value and use that value in aggregation
+        msrValues[i] = measureDefaultValue[i];
+      }
+    }
+  }
+
+  private Object getMeasureData(MeasureColumnDataChunk dataChunk, int index, DataType dataType) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      Object msrVal;
+      switch (dataType) {
+        case LONG:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+          break;
+        case DECIMAL:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+          break;
+        default:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+      }
+      return DataTypeUtil.getMeasureDataBasedOnDataType(msrVal, dataType);
+    }
+    return null;
+  }
+
+  /**
+   * Below method will used to get the result
+   */
+  @Override public Result getCollectedResult() {
+    Result<List<ListBasedResultWrapper>, Object> result = new ListBasedResult();
+    if (!tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
+      updateKeyWithLatestBlockKeyGenerator();
+    }
+    result.addScannedResult(listBasedResult);
+    return result;
+  }
+
+
+
+  /**
+   * Below method will be used to update the fixed length key with the
+   * latest block key generator
+   *
+   * @return updated block
+   */
+  private void updateKeyWithLatestBlockKeyGenerator() {
+    try {
+      long[] data = null;
+      ByteArrayWrapper key = null;
+      for (int i = 0; i < listBasedResult.size(); i++) {
+        // get the key
+        key = listBasedResult.get(i).getKey();
+        // unpack the key with table block key generator
+        data = tableBlockExecutionInfos.getBlockKeyGenerator()
+            .getKeyArray(key.getDictionaryKey(), tableBlockExecutionInfos.getMaskedByteForBlock());
+        // packed the key with latest block key generator
+        // and generate the masked key for that key
+        key.setDictionaryKey(QueryUtil
+            .getMaskedKey(restructureInfos.getKeyGenerator().generateKey(data),
+                restructureInfos.getMaxKey(), restructureInfos.getMaskByteRanges(),
+                restructureInfos.getMaskByteRanges().length));
+        listBasedResult.get(i).setKey(key);
+      }
+    } catch (KeyGenException e) {
+      LOGGER.error(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
new file mode 100644
index 0000000..6e04714
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.executor;
+
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.model.QueryModel;
+
+/**
+ * Interface for carbon query executor.
+ * Will be used to execute the query based on the query model
+ * and will return the iterator over query result
+ */
+public interface QueryExecutor<E> {
+
+  /**
+   * Below method will be used to execute the query based on query model passed from driver
+   *
+   * @param queryModel query details
+   * @return query result iterator
+   * @throws QueryExecutionException if any failure while executing the query
+   */
+  CarbonIterator<E> execute(QueryModel queryModel) throws QueryExecutionException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java b/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.java
new file mode 100644
index 0000000..724b8b6
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/QueryExecutorFactory.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.carbondata.scan.executor;
+
+import org.carbondata.scan.executor.impl.DetailQueryExecutor;
+import org.carbondata.scan.executor.impl.DetailRawRecordQueryExecutor;
+import org.carbondata.scan.model.QueryModel;
+
+/**
+ * Factory class to get the query executor from RDD
+ * This will return the executor based on query type
+ */
+public class QueryExecutorFactory {
+
+  public static QueryExecutor getQueryExecutor(QueryModel queryModel) {
+    if (queryModel.isForcedDetailRawQuery()) {
+      return new DetailRawRecordQueryExecutor();
+    } else {
+      return new DetailQueryExecutor();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/exception/QueryExecutionException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/exception/QueryExecutionException.java b/core/src/main/java/org/carbondata/scan/executor/exception/QueryExecutionException.java
new file mode 100644
index 0000000..f5d0e81
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/exception/QueryExecutionException.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.carbondata.scan.executor.exception;
+
+import java.util.Locale;
+
+/**
+ * Exception class for query execution
+ *
+ * @author Administrator
+ */
+public class QueryExecutionException extends Exception {
+
+  /**
+   * default serial version ID.
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * The Error message.
+   */
+  private String msg = "";
+
+  /**
+   * Constructor
+   *
+   * @param errorCode The error code for this exception.
+   * @param msg       The error message for this exception.
+   */
+  public QueryExecutionException(String msg) {
+    super(msg);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param errorCode The error code for this exception.
+   * @param msg       The error message for this exception.
+   */
+  public QueryExecutionException(String msg, Throwable t) {
+    super(msg, t);
+    this.msg = msg;
+  }
+
+  /**
+   * Constructor
+   *
+   * @param t
+   */
+  public QueryExecutionException(Throwable t) {
+    super(t);
+  }
+
+  /**
+   * This method is used to get the localized message.
+   *
+   * @param locale - A Locale object represents a specific geographical,
+   *               political, or cultural region.
+   * @return - Localized error message.
+   */
+  public String getLocalizedMessage(Locale locale) {
+    return "";
+  }
+
+  /**
+   * getLocalizedMessage
+   */
+  @Override public String getLocalizedMessage() {
+    return super.getLocalizedMessage();
+  }
+
+  /**
+   * getMessage
+   */
+  public String getMessage() {
+    return this.msg;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java
new file mode 100644
index 0000000..eb2261d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/AbstractQueryExecutor.java
@@ -0,0 +1,403 @@
+/*
+ * 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.carbondata.scan.executor.impl;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.HashSet;
+import java.util.List;
+import java.util.Set;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.common.logging.impl.StandardLogService;
+import org.carbondata.core.carbon.datastore.BlockIndexStore;
+import org.carbondata.core.carbon.datastore.IndexKey;
+import org.carbondata.core.carbon.datastore.block.AbstractIndex;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.datastore.exception.IndexBuilderException;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.core.keygenerator.KeyGenerator;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.executor.QueryExecutor;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.AggregatorInfo;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.infos.SortInfo;
+import org.carbondata.scan.executor.util.QueryUtil;
+import org.carbondata.scan.executor.util.RestructureUtil;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.model.QueryModel;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+/**
+ * This class provides a skeletal implementation of the {@link QueryExecutor}
+ * interface to minimize the effort required to implement this interface. This
+ * will be used to prepare all the properties required for query execution
+ */
+public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractQueryExecutor.class.getName());
+  /**
+   * holder for query properties which will be used to execute the query
+   */
+  protected QueryExecutorProperties queryProperties;
+
+  public AbstractQueryExecutor() {
+    queryProperties = new QueryExecutorProperties();
+  }
+
+  /**
+   * Below method will be used to fill the executor properties based on query
+   * model it will parse the query model and get the detail and fill it in
+   * query properties
+   *
+   * @param queryModel
+   */
+  protected void initQuery(QueryModel queryModel) throws QueryExecutionException {
+    StandardLogService.setThreadName(StandardLogService.getPartitionID(
+        queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName()),
+        queryModel.getQueryId());
+    LOGGER.info("Query will be executed on table: " + queryModel.getAbsoluteTableIdentifier()
+        .getCarbonTableIdentifier().getTableName());
+
+    QueryUtil.resolveQueryModel(queryModel);
+
+    // get the table blocks
+    try {
+      queryProperties.dataBlocks = BlockIndexStore.getInstance()
+          .loadAndGetBlocks(queryModel.getTableBlockInfos(),
+              queryModel.getAbsoluteTableIdentifier());
+    } catch (IndexBuilderException e) {
+      throw new QueryExecutionException(e);
+    }
+    //
+    // // updating the restructuring infos for the query
+    queryProperties.keyStructureInfo = getKeyStructureInfo(queryModel,
+        queryProperties.dataBlocks.get(queryProperties.dataBlocks.size() - 1).getSegmentProperties()
+            .getDimensionKeyGenerator());
+
+    // calculating the total number of aggeragted columns
+    int aggTypeCount = queryModel.getQueryMeasures().size();
+
+    int currentIndex = 0;
+    String[] aggTypes = new String[aggTypeCount];
+    DataType[] dataTypes = new DataType[aggTypeCount];
+
+    for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
+      // adding the data type and aggregation type of all the measure this
+      // can be used
+      // to select the aggregator
+      aggTypes[currentIndex] = carbonMeasure.getAggregateFunction();
+      dataTypes[currentIndex] = carbonMeasure.getMeasure().getDataType();
+      currentIndex++;
+    }
+    queryProperties.measureDataTypes = dataTypes;
+    // as aggregation will be executed in following order
+    // 1.aggregate dimension expression
+    // 2. expression
+    // 3. query measure
+    // so calculating the index of the expression start index
+    // and measure column start index
+    queryProperties.aggExpressionStartIndex = queryModel.getQueryMeasures().size();
+    queryProperties.measureStartIndex = aggTypes.length - queryModel.getQueryMeasures().size();
+
+    // dictionary column unique column id to dictionary mapping
+    // which will be used to get column actual data
+    queryProperties.columnToDictionayMapping = QueryUtil
+        .getDimensionDictionaryDetail(queryModel.getQueryDimension(),
+            queryModel.getAbsoluteTableIdentifier());
+    queryModel.setColumnToDictionaryMapping(queryProperties.columnToDictionayMapping);
+    // setting the sort dimension index. as it will be updated while getting the sort info
+    // so currently setting it to default 0 means sort is not present in any dimension
+    queryProperties.sortDimIndexes = new byte[queryModel.getQueryDimension().size()];
+  }
+
+  /**
+   * Below method will be used to get the key structure info for the uqery
+   *
+   * @param queryModel   query model
+   * @param keyGenerator
+   * @return key structure info
+   */
+  private KeyStructureInfo getKeyStructureInfo(QueryModel queryModel, KeyGenerator keyGenerator) {
+    // getting the masked byte range for dictionary column
+    int[] maskByteRanges =
+        QueryUtil.getMaskedByteRange(queryModel.getQueryDimension(), keyGenerator);
+
+    // getting the masked bytes for query dimension dictionary column
+    int[] maskedBytes = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
+
+    // max key for the dictionary dimension present in the query
+    byte[] maxKey = null;
+    try {
+      // getting the max key which will be used to masked and get the
+      // masked key
+      maxKey = QueryUtil.getMaxKeyBasedOnDimensions(queryModel.getQueryDimension(), keyGenerator);
+    } catch (KeyGenException e) {
+      LOGGER.error(e, "problem while getting the max key");
+    }
+
+    KeyStructureInfo restructureInfos = new KeyStructureInfo();
+    restructureInfos.setKeyGenerator(keyGenerator);
+    restructureInfos.setMaskByteRanges(maskByteRanges);
+    restructureInfos.setMaskedBytes(maskedBytes);
+    restructureInfos.setMaxKey(maxKey);
+    return restructureInfos;
+  }
+
+  protected List<BlockExecutionInfo> getBlockExecutionInfos(QueryModel queryModel)
+      throws QueryExecutionException {
+    initQuery(queryModel);
+    List<BlockExecutionInfo> blockExecutionInfoList = new ArrayList<BlockExecutionInfo>();
+    // fill all the block execution infos for all the blocks selected in
+    // query
+    // and query will be executed based on that infos
+    for (int i = 0; i < queryProperties.dataBlocks.size(); i++) {
+      blockExecutionInfoList
+          .add(getBlockExecutionInfoForBlock(queryModel, queryProperties.dataBlocks.get(i)));
+    }
+    return blockExecutionInfoList;
+  }
+
+  /**
+   * Below method will be used to get the block execution info which is
+   * required to execute any block  based on query model
+   *
+   * @param queryModel query model from user query
+   * @param blockIndex block index
+   * @return block execution info
+   * @throws QueryExecutionException any failure during block info creation
+   */
+  protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
+      AbstractIndex blockIndex) throws QueryExecutionException {
+    BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
+    SegmentProperties segmentProperties = blockIndex.getSegmentProperties();
+    List<CarbonDimension> tableBlockDimensions = segmentProperties.getDimensions();
+    KeyGenerator blockKeyGenerator = segmentProperties.getDimensionKeyGenerator();
+
+    // below is to get only those dimension in query which is present in the
+    // table block
+    List<QueryDimension> updatedQueryDimension = RestructureUtil
+        .getUpdatedQueryDimension(queryModel.getQueryDimension(), tableBlockDimensions);
+    // TODO add complex dimension children
+    int[] maskByteRangesForBlock =
+        QueryUtil.getMaskedByteRange(updatedQueryDimension, blockKeyGenerator);
+    int[] maksedByte =
+        QueryUtil.getMaskedByte(blockKeyGenerator.getKeySizeInBytes(), maskByteRangesForBlock);
+    blockExecutionInfo.setDimensionsExistInQuery(updatedQueryDimension.size() > 0);
+    blockExecutionInfo.setDataBlock(blockIndex);
+    blockExecutionInfo.setBlockKeyGenerator(blockKeyGenerator);
+    // adding aggregation info for query
+    blockExecutionInfo.setAggregatorInfo(getAggregatorInfoForBlock(queryModel, blockIndex));
+
+    // setting the limit
+    blockExecutionInfo.setLimit(queryModel.getLimit());
+    // setting whether detail query or not
+    blockExecutionInfo.setDetailQuery(queryModel.isDetailQuery());
+    // setting whether raw record query or not
+    blockExecutionInfo.setRawRecordDetailQuery(queryModel.isForcedDetailRawQuery());
+    // setting the masked byte of the block which will be
+    // used to update the unpack the older block keys
+    blockExecutionInfo.setMaskedByteForBlock(maksedByte);
+    // total number dimension
+    blockExecutionInfo
+        .setTotalNumberDimensionBlock(segmentProperties.getDimensionOrdinalToBlockMapping().size());
+    blockExecutionInfo
+        .setTotalNumberOfMeasureBlock(segmentProperties.getMeasuresOrdinalToBlockMapping().size());
+    // to check whether older block key update is required or not
+    blockExecutionInfo.setFixedKeyUpdateRequired(
+        blockKeyGenerator.equals(queryProperties.keyStructureInfo.getKeyGenerator()));
+    IndexKey startIndexKey = null;
+    IndexKey endIndexKey = null;
+    if (null != queryModel.getFilterExpressionResolverTree()) {
+      // loading the filter executer tree for filter evaluation
+      blockExecutionInfo.setFilterExecuterTree(FilterUtil
+          .getFilterExecuterTree(queryModel.getFilterExpressionResolverTree(), segmentProperties));
+      List<IndexKey> listOfStartEndKeys = new ArrayList<IndexKey>(2);
+      FilterUtil.traverseResolverTreeAndGetStartAndEndKey(segmentProperties,
+          queryModel.getAbsoluteTableIdentifier(), queryModel.getFilterExpressionResolverTree(),
+          listOfStartEndKeys);
+      startIndexKey = listOfStartEndKeys.get(0);
+      endIndexKey = listOfStartEndKeys.get(1);
+    } else {
+      try {
+        startIndexKey = FilterUtil.prepareDefaultStartIndexKey(segmentProperties);
+        endIndexKey = FilterUtil.prepareDefaultEndIndexKey(segmentProperties);
+      } catch (KeyGenException e) {
+        throw new QueryExecutionException(e);
+      }
+    }
+    blockExecutionInfo.setFileType(
+        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
+    //setting the start index key of the block node
+    blockExecutionInfo.setStartKey(startIndexKey);
+    //setting the end index key of the block node
+    blockExecutionInfo.setEndKey(endIndexKey);
+    // expression dimensions
+    List<CarbonDimension> expressionDimensions =
+        new ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    // expression measure
+    List<CarbonMeasure> expressionMeasures =
+        new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    // setting all the dimension chunk indexes to be read from file
+    blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(QueryUtil
+        .getDimensionsBlockIndexes(updatedQueryDimension,
+            segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions));
+    // setting all the measure chunk indexes to be read from file
+    blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(QueryUtil
+        .getMeasureBlockIndexes(queryModel.getQueryMeasures(), expressionMeasures,
+            segmentProperties.getMeasuresOrdinalToBlockMapping()));
+    // setting the key structure info which will be required
+    // to update the older block key with new key generator
+    blockExecutionInfo.setKeyStructureInfo(queryProperties.keyStructureInfo);
+    // setting the size of fixed key column (dictionary column)
+    blockExecutionInfo.setFixedLengthKeySize(getKeySize(updatedQueryDimension, segmentProperties));
+    Set<Integer> dictionaryColumnBlockIndex = new HashSet<Integer>();
+    List<Integer> noDictionaryColumnBlockIndex = new ArrayList<Integer>();
+    // get the block index to be read from file for query dimension
+    // for both dictionary columns and no dictionary columns
+    QueryUtil.fillQueryDimensionsBlockIndexes(updatedQueryDimension,
+        segmentProperties.getDimensionOrdinalToBlockMapping(), dictionaryColumnBlockIndex,
+        noDictionaryColumnBlockIndex);
+    int[] queryDictionaruColumnBlockIndexes = ArrayUtils.toPrimitive(
+        dictionaryColumnBlockIndex.toArray(new Integer[dictionaryColumnBlockIndex.size()]));
+    // need to sort the dictionary column as for all dimension
+    // column key will be filled based on key order
+    Arrays.sort(queryDictionaruColumnBlockIndexes);
+    blockExecutionInfo.setDictionaryColumnBlockIndex(queryDictionaruColumnBlockIndexes);
+    // setting the no dictionary column block indexes
+    blockExecutionInfo.setNoDictionaryBlockIndexes(ArrayUtils.toPrimitive(
+        noDictionaryColumnBlockIndex.toArray(new Integer[noDictionaryColumnBlockIndex.size()])));
+    // setting column id to dictionary mapping
+    blockExecutionInfo.setColumnIdToDcitionaryMapping(queryProperties.columnToDictionayMapping);
+    // setting each column value size
+    blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize());
+    try {
+      // to set column group and its key structure info which will be used
+      // to
+      // for getting the column group column data in case of final row
+      // and in case of dimension aggregation
+      blockExecutionInfo.setColumnGroupToKeyStructureInfo(
+          QueryUtil.getColumnGroupKeyStructureInfo(updatedQueryDimension, segmentProperties));
+    } catch (KeyGenException e) {
+      throw new QueryExecutionException(e);
+    }
+    return blockExecutionInfo;
+  }
+
+  /**
+   * This method will be used to get fixed key length size this will be used
+   * to create a row from column chunk
+   *
+   * @param queryDimension    query dimension
+   * @param blockMetadataInfo block metadata info
+   * @return key size
+   */
+  private int getKeySize(List<QueryDimension> queryDimension, SegmentProperties blockMetadataInfo) {
+    List<Integer> fixedLengthDimensionOrdinal =
+        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    int counter = 0;
+    while (counter < queryDimension.size()) {
+      if (queryDimension.get(counter).getDimension().numberOfChild() > 0) {
+        counter += queryDimension.get(counter).getDimension().numberOfChild();
+        continue;
+      } else if (!CarbonUtil.hasEncoding(queryDimension.get(counter).getDimension().getEncoder(),
+          Encoding.DICTIONARY)) {
+        counter++;
+      } else {
+        fixedLengthDimensionOrdinal.add(queryDimension.get(counter).getDimension().getKeyOrdinal());
+        counter++;
+      }
+    }
+    int[] dictioanryColumnOrdinal = ArrayUtils.toPrimitive(
+        fixedLengthDimensionOrdinal.toArray(new Integer[fixedLengthDimensionOrdinal.size()]));
+    if (dictioanryColumnOrdinal.length > 0) {
+      return blockMetadataInfo.getFixedLengthKeySplitter()
+          .getKeySizeByBlock(dictioanryColumnOrdinal);
+    }
+    return 0;
+  }
+
+  /**
+   * Below method will be used to get the sort information which will be
+   * required during sorting the data on dimension column
+   *
+   * @param queryModel query model
+   * @return Sort infos
+   * @throws QueryExecutionException if problem while
+   */
+  protected SortInfo getSortInfos(QueryModel queryModel) throws QueryExecutionException {
+
+    // get the masked by range for order by dimension
+    int[][] maskedByteRangeForSorting = QueryUtil
+        .getMaskedByteRangeForSorting(queryModel.getSortDimension(),
+            queryProperties.keyStructureInfo.getKeyGenerator(),
+            queryProperties.keyStructureInfo.getMaskByteRanges());
+    // get masked key for sorting
+    byte[][] maksedKeyForSorting = QueryUtil.getMaksedKeyForSorting(queryModel.getSortDimension(),
+        queryProperties.keyStructureInfo.getKeyGenerator(), maskedByteRangeForSorting,
+        queryProperties.keyStructureInfo.getMaskByteRanges());
+    // fill sort dimension indexes
+    queryProperties.sortDimIndexes = QueryUtil
+        .getSortDimensionIndexes(queryModel.getSortDimension(), queryModel.getQueryDimension());
+    SortInfo sortInfos = new SortInfo();
+    sortInfos.setDimensionMaskKeyForSorting(maksedKeyForSorting);
+    sortInfos.setDimensionSortOrder(queryModel.getSortOrder());
+    sortInfos.setMaskedByteRangeForSorting(maskedByteRangeForSorting);
+    sortInfos.setSortDimensionIndex(queryProperties.sortDimIndexes);
+    sortInfos.setSortDimension(queryModel.getSortDimension());
+    return sortInfos;
+  }
+
+  /**
+   * Below method will be used to get the aggrgator info for the query
+   *
+   * @param queryModel query model
+   * @param tableBlock table block
+   * @return aggregator info
+   */
+  private AggregatorInfo getAggregatorInfoForBlock(QueryModel queryModel,
+      AbstractIndex tableBlock) {
+    // getting the aggregate infos which will be used during aggregation
+    AggregatorInfo aggregatorInfos = RestructureUtil
+        .getAggregatorInfos(queryModel.getQueryMeasures(),
+            tableBlock.getSegmentProperties().getMeasures());
+    // setting the index of expression in measure aggregators
+    aggregatorInfos.setExpressionAggregatorStartIndex(queryProperties.aggExpressionStartIndex);
+    // setting the index of measure columns in measure aggregators
+    aggregatorInfos.setMeasureAggregatorStartIndex(queryProperties.measureStartIndex);
+    // setting the measure aggregator for all aggregation function selected
+    // in query
+    aggregatorInfos.setMeasureDataTypes(queryProperties.measureDataTypes);
+    return aggregatorInfos;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
new file mode 100644
index 0000000..8232567
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
@@ -0,0 +1,46 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.executor.impl;
+
+import java.util.List;
+
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.result.iterator.ChunkRowIterator;
+import org.carbondata.scan.result.iterator.DetailQueryResultIterator;
+import org.carbondata.scan.result.preparator.impl.DetailQueryResultPreparatorImpl;
+
+/**
+ * Below class will be used to execute the detail query
+ * For executing the detail query it will pass all the block execution
+ * info to detail query result iterator and iterator will be returned
+ */
+public class DetailQueryExecutor extends AbstractQueryExecutor {
+
+  @Override public CarbonIterator<Object[]> execute(QueryModel queryModel)
+      throws QueryExecutionException {
+    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
+    return new ChunkRowIterator(
+        new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
+            new DetailQueryResultPreparatorImpl(queryProperties, queryModel)));
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
new file mode 100644
index 0000000..1ce0a36
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
@@ -0,0 +1,24 @@
+package org.carbondata.scan.executor.impl;
+
+import java.util.List;
+
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.result.BatchResult;
+import org.carbondata.scan.result.iterator.DetailQueryResultIterator;
+import org.carbondata.scan.result.preparator.impl.RawQueryResultPreparatorImpl;
+
+/**
+ * Executor for raw records, it does not parse to actual data
+ */
+public class DetailRawRecordQueryExecutor extends AbstractQueryExecutor<BatchResult> {
+
+  @Override public CarbonIterator<BatchResult> execute(QueryModel queryModel)
+      throws QueryExecutionException {
+    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
+    return new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
+        new RawQueryResultPreparatorImpl(queryProperties, queryModel));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/impl/QueryExecutorProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/QueryExecutorProperties.java b/core/src/main/java/org/carbondata/scan/executor/impl/QueryExecutorProperties.java
new file mode 100644
index 0000000..a004dce
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/QueryExecutorProperties.java
@@ -0,0 +1,78 @@
+/*
+ * 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.carbondata.scan.executor.impl;
+
+import java.util.List;
+import java.util.Map;
+
+import org.carbondata.core.cache.dictionary.Dictionary;
+import org.carbondata.core.carbon.datastore.block.AbstractIndex;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
+
+/**
+ * Holds all the properties required for query execution
+ */
+public class QueryExecutorProperties {
+
+  /**
+   * list of blocks in which query will be executed
+   */
+  protected List<AbstractIndex> dataBlocks;
+
+  /**
+   * holds the information required for updating the order block
+   * dictionary key
+   */
+  public KeyStructureInfo keyStructureInfo;
+
+  /**
+   * as we have multiple type of column aggregation like
+   * dimension,expression,measure so this will be used to for getting the
+   * measure aggregation start index
+   */
+  public int measureStartIndex;
+
+  /**
+   * query like count(1),count(*) ,etc will used this parameter
+   */
+  public boolean isFunctionQuery;
+
+  /**
+   * aggExpressionStartIndex
+   */
+  public int aggExpressionStartIndex;
+
+  /**
+   * index of the dimension which is present in the order by
+   * in a query
+   */
+  public byte[] sortDimIndexes;
+
+  /**
+   * this will hold the information about the dictionary dimension
+   * which to
+   */
+  public Map<String, Dictionary> columnToDictionayMapping;
+
+  /**
+   * Measure datatypes
+   */
+  public DataType[] measureDataTypes;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/infos/AggregatorInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/infos/AggregatorInfo.java b/core/src/main/java/org/carbondata/scan/executor/infos/AggregatorInfo.java
new file mode 100644
index 0000000..2c163e1
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/infos/AggregatorInfo.java
@@ -0,0 +1,149 @@
+/*
+ * 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.carbondata.scan.executor.infos;
+
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+
+/**
+ * Info class which store all the details
+ * which is required during aggregation
+ */
+public class AggregatorInfo {
+
+  /**
+   * selected query measure ordinal
+   * which will be used to read the measures chunk data
+   * this will be storing the index of the measure in measures chunk
+   */
+  private int[] measureOrdinals;
+
+  /**
+   * This parameter will be used to
+   * check whether particular measure is present
+   * in the table block, if not then its default value will be used
+   */
+  private boolean[] measureExists;
+
+  /**
+   * this default value will be used to when some measure is not present
+   * in the table block, in case of restructuring of the table if user is adding any
+   * measure then in older block that measure wont be present so for measure default value
+   * will be used to aggregate in the older table block query execution
+   */
+  private Object[] defaultValues;
+
+  /**
+   * In carbon there are three type of aggregation
+   * (dimension aggregation, expression aggregation and measure aggregation)
+   * Below index will be used to set the start position of expression in measures
+   * aggregator array
+   */
+  private int expressionAggregatorStartIndex;
+
+  /**
+   * In carbon there are three type of aggregation
+   * (dimension aggregation, expression aggregation and measure aggregation)
+   * Below index will be used to set the start position of measures in measures
+   * aggregator array
+   */
+  private int measureAggregatorStartIndex;
+
+  /**
+   * Datatype of each measure;
+   */
+  private DataType[] measureDataTypes;
+
+  /**
+   * @return the measureOrdinal
+   */
+  public int[] getMeasureOrdinals() {
+    return measureOrdinals;
+  }
+
+  /**
+   * @param measureOrdinal the measureOrdinal to set
+   */
+  public void setMeasureOrdinals(int[] measureOrdinal) {
+    this.measureOrdinals = measureOrdinal;
+  }
+
+  /**
+   * @return the measureExists
+   */
+  public boolean[] getMeasureExists() {
+    return measureExists;
+  }
+
+  /**
+   * @param measureExists the measureExists to set
+   */
+  public void setMeasureExists(boolean[] measureExists) {
+    this.measureExists = measureExists;
+  }
+
+  /**
+   * @return the defaultValues
+   */
+  public Object[] getDefaultValues() {
+    return defaultValues;
+  }
+
+  /**
+   * @param defaultValues the defaultValues to set
+   */
+  public void setDefaultValues(Object[] defaultValues) {
+    this.defaultValues = defaultValues;
+  }
+
+  /**
+   * @return the expressionAggregatorStartIndex
+   */
+  public int getExpressionAggregatorStartIndex() {
+    return expressionAggregatorStartIndex;
+  }
+
+  /**
+   * @param expressionAggregatorStartIndex the expressionAggregatorStartIndex to set
+   */
+  public void setExpressionAggregatorStartIndex(int expressionAggregatorStartIndex) {
+    this.expressionAggregatorStartIndex = expressionAggregatorStartIndex;
+  }
+
+  /**
+   * @return the measureAggregatorStartIndex
+   */
+  public int getMeasureAggregatorStartIndex() {
+    return measureAggregatorStartIndex;
+  }
+
+  /**
+   * @param measureAggregatorStartIndex the measureAggregatorStartIndex to set
+   */
+  public void setMeasureAggregatorStartIndex(int measureAggregatorStartIndex) {
+    this.measureAggregatorStartIndex = measureAggregatorStartIndex;
+  }
+
+  public DataType[] getMeasureDataTypes() {
+    return measureDataTypes;
+  }
+
+  public void setMeasureDataTypes(DataType[] measureDataTypes) {
+    this.measureDataTypes = measureDataTypes;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java
new file mode 100644
index 0000000..dc55e46
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/infos/BlockExecutionInfo.java
@@ -0,0 +1,611 @@
+/*
+ * 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.carbondata.scan.executor.infos;
+
+import java.util.Map;
+
+import org.carbondata.core.cache.dictionary.Dictionary;
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.carbon.datastore.IndexKey;
+import org.carbondata.core.carbon.datastore.block.AbstractIndex;
+import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
+import org.carbondata.core.keygenerator.KeyGenerator;
+import org.carbondata.scan.filter.executer.FilterExecuter;
+
+/**
+ * Below class will have all the properties which needed during query execution
+ * for one block
+ */
+public class BlockExecutionInfo {
+
+  /**
+   * block on which query will be executed
+   */
+  private AbstractIndex blockIndex;
+
+  /**
+   * each segment key size can be different and in that case we need to update
+   * the fixed key with latest segment key generator. so this property will
+   * tell whether this is required or not if key size is same then it is not
+   * required
+   */
+  private boolean isFixedKeyUpdateRequired;
+
+  /**
+   * in case of detail+order by query when number of output record is same we
+   * need to store data in the disk, so for this check will be used to whether
+   * we can write in the disk or not
+   */
+  private boolean isFileBasedQuery;
+
+  /**
+   * id of the query. this will be used to create directory while writing the
+   * data file in case of detail+order by query
+   */
+  private String queryId;
+
+  /**
+   * this to handle limit query in case of detail query we are pushing down
+   * the limit to executor level so based on the number of limit we can
+   * process only that many records
+   */
+  private int limit;
+
+  /**
+   * below to store all the information required for aggregation during query
+   * execution
+   */
+  private AggregatorInfo aggregatorInfo;
+
+  /**
+   * this will be used to get the first tentative block from which query
+   * execution start, this will be useful in case of filter query to get the
+   * start block based on filter values
+   */
+  private IndexKey startKey;
+
+  /**
+   * this will be used to get the last tentative block till which scanning
+   * will be done, this will be useful in case of filter query to get the last
+   * block based on filter values
+   */
+  private IndexKey endKey;
+
+  /**
+   * masked byte for block which will be used to unpack the fixed length key,
+   * this will be used for updating the older block key with new block key
+   * generator
+   */
+  private int[] maskedByteForBlock;
+
+  /**
+   * flag to check whether query is detail query or aggregation query
+   */
+  private boolean isDetailQuery;
+
+  /**
+   * total number of dimension in block
+   */
+  private int totalNumberDimensionBlock;
+
+  /**
+   * total number of measure in block
+   */
+  private int totalNumberOfMeasureBlock;
+
+  /**
+   * will be used to read the dimension block from file
+   */
+  private int[] allSelectedDimensionBlocksIndexes;
+
+  /**
+   * will be used to read the measure block from file
+   */
+  private int[] allSelectedMeasureBlocksIndexes;
+
+  /**
+   * this will be used to update the older block fixed length keys with the
+   * new block fixed length key
+   */
+  private KeyStructureInfo keyStructureInfo;
+
+  /**
+   * below will be used to sort the data based
+   */
+  private SortInfo sortInfo;
+
+  /**
+   * first block from which query execution will start
+   */
+  private DataRefNode firstDataBlock;
+
+  /**
+   * number of block to be scanned in the query
+   */
+  private long numberOfBlockToScan;
+
+  /**
+   * key size of the fixed length dimension column
+   */
+  private int fixedLengthKeySize;
+
+  /**
+   * dictionary column block indexes based on query
+   */
+  private int[] dictionaryColumnBlockIndex;
+  /**
+   * no dictionary column block indexes in based on the query order
+   */
+  private int[] noDictionaryBlockIndexes;
+
+  /**
+   * key generator used for generating the table block fixed length key
+   */
+  private KeyGenerator blockKeyGenerator;
+
+  /**
+   * each column value size
+   */
+  private int[] eachColumnValueSize;
+
+  /**
+   * partition number
+   */
+  private String partitionId;
+
+  /**
+   * column group block index in file to key structure info mapping
+   */
+  private Map<Integer, KeyStructureInfo> columnGroupToKeyStructureInfo;
+
+  /**
+   * mapping of dictionary dimension to its dictionary mapping which will be
+   * used to get the actual data from dictionary for aggregation, sorting
+   */
+  private Map<String, Dictionary> columnIdToDcitionaryMapping;
+
+  /**
+   * filter tree to execute the filter
+   */
+  private FilterExecuter filterExecuterTree;
+
+  /**
+   * fileType
+   */
+  private FileType fileType;
+
+  /**
+   * whether it needs only raw byte records with out aggregation.
+   */
+  private boolean isRawRecordDetailQuery;
+
+  /**
+   * whether dimensions exist in query.
+   */
+  private boolean isDimensionsExistInQuery;
+
+  /**
+   * @return the tableBlock
+   */
+  public AbstractIndex getDataBlock() {
+    return blockIndex;
+  }
+
+  /**
+   * @param blockIndex the tableBlock to set
+   */
+  public void setDataBlock(AbstractIndex blockIndex) {
+    this.blockIndex = blockIndex;
+  }
+
+  /**
+   * @return the isFixedKeyUpdateRequired
+   */
+  public boolean isFixedKeyUpdateRequired() {
+    return isFixedKeyUpdateRequired;
+  }
+
+  /**
+   * @param isFixedKeyUpdateRequired the isFixedKeyUpdateRequired to set
+   */
+  public void setFixedKeyUpdateRequired(boolean isFixedKeyUpdateRequired) {
+    this.isFixedKeyUpdateRequired = isFixedKeyUpdateRequired;
+  }
+
+  /**
+   * @return the isFileBasedQuery
+   */
+  public boolean isFileBasedQuery() {
+    return isFileBasedQuery;
+  }
+
+  /**
+   * @param isFileBasedQuery the isFileBasedQuery to set
+   */
+  public void setFileBasedQuery(boolean isFileBasedQuery) {
+    this.isFileBasedQuery = isFileBasedQuery;
+  }
+
+  /**
+   * @return the queryId
+   */
+  public String getQueryId() {
+    return queryId;
+  }
+
+  /**
+   * @param queryId the queryId to set
+   */
+  public void setQueryId(String queryId) {
+    this.queryId = queryId;
+  }
+
+  /**
+   * @return the limit
+   */
+  public int getLimit() {
+    return limit;
+  }
+
+  /**
+   * @param limit the limit to set
+   */
+  public void setLimit(int limit) {
+    this.limit = limit;
+  }
+
+  /**
+   * @return the aggregatorInfos
+   */
+  public AggregatorInfo getAggregatorInfo() {
+    return aggregatorInfo;
+  }
+
+  /**
+   * @param aggregatorInfo the aggregatorInfos to set
+   */
+  public void setAggregatorInfo(AggregatorInfo aggregatorInfo) {
+    this.aggregatorInfo = aggregatorInfo;
+  }
+
+  /**
+   * @return the startKey
+   */
+  public IndexKey getStartKey() {
+    return startKey;
+  }
+
+  /**
+   * @param startKey the startKey to set
+   */
+  public void setStartKey(IndexKey startKey) {
+    this.startKey = startKey;
+  }
+
+  /**
+   * @return the endKey
+   */
+  public IndexKey getEndKey() {
+    return endKey;
+  }
+
+  /**
+   * @param endKey the endKey to set
+   */
+  public void setEndKey(IndexKey endKey) {
+    this.endKey = endKey;
+  }
+
+  /**
+   * @return the maskedByteForBlock
+   */
+  public int[] getMaskedByteForBlock() {
+    return maskedByteForBlock;
+  }
+
+  /**
+   * @param maskedByteForBlock the maskedByteForBlock to set
+   */
+  public void setMaskedByteForBlock(int[] maskedByteForBlock) {
+    this.maskedByteForBlock = maskedByteForBlock;
+  }
+
+  /**
+   * @return the isDetailQuery
+   */
+  public boolean isDetailQuery() {
+    return isDetailQuery;
+  }
+
+  /**
+   * @param isDetailQuery the isDetailQuery to set
+   */
+  public void setDetailQuery(boolean isDetailQuery) {
+    this.isDetailQuery = isDetailQuery;
+  }
+
+  /**
+   * @return the totalNumberDimensionBlock
+   */
+  public int getTotalNumberDimensionBlock() {
+    return totalNumberDimensionBlock;
+  }
+
+  /**
+   * @param totalNumberDimensionBlock the totalNumberDimensionBlock to set
+   */
+  public void setTotalNumberDimensionBlock(int totalNumberDimensionBlock) {
+    this.totalNumberDimensionBlock = totalNumberDimensionBlock;
+  }
+
+  /**
+   * @return the totalNumberOfMeasureBlock
+   */
+  public int getTotalNumberOfMeasureBlock() {
+    return totalNumberOfMeasureBlock;
+  }
+
+  /**
+   * @param totalNumberOfMeasureBlock the totalNumberOfMeasureBlock to set
+   */
+  public void setTotalNumberOfMeasureBlock(int totalNumberOfMeasureBlock) {
+    this.totalNumberOfMeasureBlock = totalNumberOfMeasureBlock;
+  }
+
+  /**
+   * @return the allSelectedDimensionBlocksIndexes
+   */
+  public int[] getAllSelectedDimensionBlocksIndexes() {
+    return allSelectedDimensionBlocksIndexes;
+  }
+
+  /**
+   * @param allSelectedDimensionBlocksIndexes the allSelectedDimensionBlocksIndexes to set
+   */
+  public void setAllSelectedDimensionBlocksIndexes(int[] allSelectedDimensionBlocksIndexes) {
+    this.allSelectedDimensionBlocksIndexes = allSelectedDimensionBlocksIndexes;
+  }
+
+  /**
+   * @return the allSelectedMeasureBlocksIndexes
+   */
+  public int[] getAllSelectedMeasureBlocksIndexes() {
+    return allSelectedMeasureBlocksIndexes;
+  }
+
+  /**
+   * @param allSelectedMeasureBlocksIndexes the allSelectedMeasureBlocksIndexes to set
+   */
+  public void setAllSelectedMeasureBlocksIndexes(int[] allSelectedMeasureBlocksIndexes) {
+    this.allSelectedMeasureBlocksIndexes = allSelectedMeasureBlocksIndexes;
+  }
+
+  /**
+   * @return the restructureInfos
+   */
+  public KeyStructureInfo getKeyStructureInfo() {
+    return keyStructureInfo;
+  }
+
+  /**
+   * @param keyStructureInfo the restructureInfos to set
+   */
+  public void setKeyStructureInfo(KeyStructureInfo keyStructureInfo) {
+    this.keyStructureInfo = keyStructureInfo;
+  }
+
+  /**
+   * @return the sortInfos
+   */
+  public SortInfo getSortInfo() {
+    return sortInfo;
+  }
+
+  /**
+   * @param sortInfo the sortInfos to set
+   */
+  public void setSortInfo(SortInfo sortInfo) {
+    this.sortInfo = sortInfo;
+  }
+
+  /**
+   * @return the firstDataBlock
+   */
+  public DataRefNode getFirstDataBlock() {
+    return firstDataBlock;
+  }
+
+  /**
+   * @param firstDataBlock the firstDataBlock to set
+   */
+  public void setFirstDataBlock(DataRefNode firstDataBlock) {
+    this.firstDataBlock = firstDataBlock;
+  }
+
+  /**
+   * @return the numberOfBlockToScan
+   */
+  public long getNumberOfBlockToScan() {
+    return numberOfBlockToScan;
+  }
+
+  /**
+   * @param numberOfBlockToScan the numberOfBlockToScan to set
+   */
+  public void setNumberOfBlockToScan(long numberOfBlockToScan) {
+    this.numberOfBlockToScan = numberOfBlockToScan;
+  }
+
+  /**
+   * @return the fixedLengthKeySize
+   */
+  public int getFixedLengthKeySize() {
+    return fixedLengthKeySize;
+  }
+
+  /**
+   * @param fixedLengthKeySize the fixedLengthKeySize to set
+   */
+  public void setFixedLengthKeySize(int fixedLengthKeySize) {
+    this.fixedLengthKeySize = fixedLengthKeySize;
+  }
+
+  /**
+   * @return the filterEvaluatorTree
+   */
+  public FilterExecuter getFilterExecuterTree() {
+    return filterExecuterTree;
+  }
+
+  /**
+   * @param filterExecuterTree the filterEvaluatorTree to set
+   */
+  public void setFilterExecuterTree(FilterExecuter filterExecuterTree) {
+    this.filterExecuterTree = filterExecuterTree;
+  }
+
+  /**
+   * @return the tableBlockKeyGenerator
+   */
+  public KeyGenerator getBlockKeyGenerator() {
+    return blockKeyGenerator;
+  }
+
+  /**
+   * @param tableBlockKeyGenerator the tableBlockKeyGenerator to set
+   */
+  public void setBlockKeyGenerator(KeyGenerator tableBlockKeyGenerator) {
+    this.blockKeyGenerator = tableBlockKeyGenerator;
+  }
+
+  /**
+   * @return the eachColumnValueSize
+   */
+  public int[] getEachColumnValueSize() {
+    return eachColumnValueSize;
+  }
+
+  /**
+   * @param eachColumnValueSize the eachColumnValueSize to set
+   */
+  public void setEachColumnValueSize(int[] eachColumnValueSize) {
+    this.eachColumnValueSize = eachColumnValueSize;
+  }
+
+  /**
+   * @return the partitionId
+   */
+  public String getPartitionId() {
+    return partitionId;
+  }
+
+  /**
+   * @param partitionId the partitionId to set
+   */
+  public void setPartitionId(String partitionId) {
+    this.partitionId = partitionId;
+  }
+
+  /**
+   * @return the dictionaryColumnBlockIndex
+   */
+  public int[] getDictionaryColumnBlockIndex() {
+    return dictionaryColumnBlockIndex;
+  }
+
+  /**
+   * @param dictionaryColumnBlockIndex the dictionaryColumnBlockIndex to set
+   */
+  public void setDictionaryColumnBlockIndex(int[] dictionaryColumnBlockIndex) {
+    this.dictionaryColumnBlockIndex = dictionaryColumnBlockIndex;
+  }
+
+  /**
+   * @return the noDictionaryBlockIndexes
+   */
+  public int[] getNoDictionaryBlockIndexes() {
+    return noDictionaryBlockIndexes;
+  }
+
+  /**
+   * @param noDictionaryBlockIndexes the noDictionaryBlockIndexes to set
+   */
+  public void setNoDictionaryBlockIndexes(int[] noDictionaryBlockIndexes) {
+    this.noDictionaryBlockIndexes = noDictionaryBlockIndexes;
+  }
+
+  /**
+   * @return the columnGroupToKeyStructureInfo
+   */
+  public Map<Integer, KeyStructureInfo> getColumnGroupToKeyStructureInfo() {
+    return columnGroupToKeyStructureInfo;
+  }
+
+  /**
+   * @param columnGroupToKeyStructureInfo the columnGroupToKeyStructureInfo to set
+   */
+  public void setColumnGroupToKeyStructureInfo(
+      Map<Integer, KeyStructureInfo> columnGroupToKeyStructureInfo) {
+    this.columnGroupToKeyStructureInfo = columnGroupToKeyStructureInfo;
+  }
+
+  /**
+   * @return the columnIdToDcitionaryMapping
+   */
+  public Map<String, Dictionary> getColumnIdToDcitionaryMapping() {
+    return columnIdToDcitionaryMapping;
+  }
+
+  /**
+   * @param columnIdToDcitionaryMapping the columnIdToDcitionaryMapping to set
+   */
+  public void setColumnIdToDcitionaryMapping(Map<String, Dictionary> columnIdToDcitionaryMapping) {
+    this.columnIdToDcitionaryMapping = columnIdToDcitionaryMapping;
+  }
+
+  /**
+   * @return the fileType
+   */
+  public FileType getFileType() {
+    return fileType;
+  }
+
+  /**
+   * @param fileType the fileType to set
+   */
+  public void setFileType(FileType fileType) {
+    this.fileType = fileType;
+  }
+
+  public boolean isRawRecordDetailQuery() {
+    return isRawRecordDetailQuery;
+  }
+
+  public void setRawRecordDetailQuery(boolean rawRecordDetailQuery) {
+    isRawRecordDetailQuery = rawRecordDetailQuery;
+  }
+
+  public boolean isDimensionsExistInQuery() {
+    return isDimensionsExistInQuery;
+  }
+
+  public void setDimensionsExistInQuery(boolean dimensionsExistInQuery) {
+    isDimensionsExistInQuery = dimensionsExistInQuery;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java b/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
new file mode 100644
index 0000000..51874b4
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.executor.infos;
+
+import org.carbondata.core.keygenerator.KeyGenerator;
+
+/**
+ * Below class will store the structure of the key
+ * used during query execution
+ */
+public class KeyStructureInfo {
+
+  /**
+   * it's actually a latest key generator
+   * last table block as this key generator will be used to
+   * to update the mdkey of the older slice with the new slice
+   */
+  private KeyGenerator keyGenerator;
+
+  /**
+   * mask bytes ranges for the query
+   */
+  private int[] maskByteRanges;
+
+  /**
+   * masked bytes of the query
+   */
+  private int[] maskedBytes;
+
+  /**
+   * max key for query execution
+   */
+  private byte[] maxKey;
+
+  /**
+   * mdkey start index of block
+   */
+  private int blockMdKeyStartOffset;
+
+  /**
+   * @return the keyGenerator
+   */
+  public KeyGenerator getKeyGenerator() {
+    return keyGenerator;
+  }
+
+  /**
+   * @param keyGenerator the keyGenerator to set
+   */
+  public void setKeyGenerator(KeyGenerator keyGenerator) {
+    this.keyGenerator = keyGenerator;
+  }
+
+  /**
+   * @return the maskByteRanges
+   */
+  public int[] getMaskByteRanges() {
+    return maskByteRanges;
+  }
+
+  /**
+   * @param maskByteRanges the maskByteRanges to set
+   */
+  public void setMaskByteRanges(int[] maskByteRanges) {
+    this.maskByteRanges = maskByteRanges;
+  }
+
+  /**
+   * @return the maskedBytes
+   */
+  public int[] getMaskedBytes() {
+    return maskedBytes;
+  }
+
+  /**
+   * @param maskedBytes the maskedBytes to set
+   */
+  public void setMaskedBytes(int[] maskedBytes) {
+    this.maskedBytes = maskedBytes;
+  }
+
+  /**
+   * @return the maxKey
+   */
+  public byte[] getMaxKey() {
+    return maxKey;
+  }
+
+  /**
+   * @param maxKey the maxKey to set
+   */
+  public void setMaxKey(byte[] maxKey) {
+    this.maxKey = maxKey;
+  }
+
+  /**
+   * @param startOffset
+   */
+  public void setBlockMdKeyStartOffset(int startOffset) {
+    this.blockMdKeyStartOffset = startOffset;
+  }
+
+  /**
+   * @return
+   */
+  public int getBlockMdKeyStartOffset() {
+    return this.blockMdKeyStartOffset;
+  }
+}



[39/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
deleted file mode 100644
index c9e9732..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
+++ /dev/null
@@ -1,149 +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.carbondata.query.carbon.executor.infos;
-
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-
-/**
- * Info class which store all the details
- * which is required during aggregation
- */
-public class AggregatorInfo {
-
-  /**
-   * selected query measure ordinal
-   * which will be used to read the measures chunk data
-   * this will be storing the index of the measure in measures chunk
-   */
-  private int[] measureOrdinals;
-
-  /**
-   * This parameter will be used to
-   * check whether particular measure is present
-   * in the table block, if not then its default value will be used
-   */
-  private boolean[] measureExists;
-
-  /**
-   * this default value will be used to when some measure is not present
-   * in the table block, in case of restructuring of the table if user is adding any
-   * measure then in older block that measure wont be present so for measure default value
-   * will be used to aggregate in the older table block query execution
-   */
-  private Object[] defaultValues;
-
-  /**
-   * In carbon there are three type of aggregation
-   * (dimension aggregation, expression aggregation and measure aggregation)
-   * Below index will be used to set the start position of expression in measures
-   * aggregator array
-   */
-  private int expressionAggregatorStartIndex;
-
-  /**
-   * In carbon there are three type of aggregation
-   * (dimension aggregation, expression aggregation and measure aggregation)
-   * Below index will be used to set the start position of measures in measures
-   * aggregator array
-   */
-  private int measureAggregatorStartIndex;
-
-  /**
-   * Datatype of each measure;
-   */
-  private DataType[] measureDataTypes;
-
-  /**
-   * @return the measureOrdinal
-   */
-  public int[] getMeasureOrdinals() {
-    return measureOrdinals;
-  }
-
-  /**
-   * @param measureOrdinal the measureOrdinal to set
-   */
-  public void setMeasureOrdinals(int[] measureOrdinal) {
-    this.measureOrdinals = measureOrdinal;
-  }
-
-  /**
-   * @return the measureExists
-   */
-  public boolean[] getMeasureExists() {
-    return measureExists;
-  }
-
-  /**
-   * @param measureExists the measureExists to set
-   */
-  public void setMeasureExists(boolean[] measureExists) {
-    this.measureExists = measureExists;
-  }
-
-  /**
-   * @return the defaultValues
-   */
-  public Object[] getDefaultValues() {
-    return defaultValues;
-  }
-
-  /**
-   * @param defaultValues the defaultValues to set
-   */
-  public void setDefaultValues(Object[] defaultValues) {
-    this.defaultValues = defaultValues;
-  }
-
-  /**
-   * @return the expressionAggregatorStartIndex
-   */
-  public int getExpressionAggregatorStartIndex() {
-    return expressionAggregatorStartIndex;
-  }
-
-  /**
-   * @param expressionAggregatorStartIndex the expressionAggregatorStartIndex to set
-   */
-  public void setExpressionAggregatorStartIndex(int expressionAggregatorStartIndex) {
-    this.expressionAggregatorStartIndex = expressionAggregatorStartIndex;
-  }
-
-  /**
-   * @return the measureAggregatorStartIndex
-   */
-  public int getMeasureAggregatorStartIndex() {
-    return measureAggregatorStartIndex;
-  }
-
-  /**
-   * @param measureAggregatorStartIndex the measureAggregatorStartIndex to set
-   */
-  public void setMeasureAggregatorStartIndex(int measureAggregatorStartIndex) {
-    this.measureAggregatorStartIndex = measureAggregatorStartIndex;
-  }
-
-  public DataType[] getMeasureDataTypes() {
-    return measureDataTypes;
-  }
-
-  public void setMeasureDataTypes(DataType[] measureDataTypes) {
-    this.measureDataTypes = measureDataTypes;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
deleted file mode 100644
index 202a932..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
+++ /dev/null
@@ -1,611 +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.carbondata.query.carbon.executor.infos;
-
-import java.util.Map;
-
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.carbon.datastore.IndexKey;
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.query.filter.executer.FilterExecuter;
-
-/**
- * Below class will have all the properties which needed during query execution
- * for one block
- */
-public class BlockExecutionInfo {
-
-  /**
-   * block on which query will be executed
-   */
-  private AbstractIndex blockIndex;
-
-  /**
-   * each segment key size can be different and in that case we need to update
-   * the fixed key with latest segment key generator. so this property will
-   * tell whether this is required or not if key size is same then it is not
-   * required
-   */
-  private boolean isFixedKeyUpdateRequired;
-
-  /**
-   * in case of detail+order by query when number of output record is same we
-   * need to store data in the disk, so for this check will be used to whether
-   * we can write in the disk or not
-   */
-  private boolean isFileBasedQuery;
-
-  /**
-   * id of the query. this will be used to create directory while writing the
-   * data file in case of detail+order by query
-   */
-  private String queryId;
-
-  /**
-   * this to handle limit query in case of detail query we are pushing down
-   * the limit to executor level so based on the number of limit we can
-   * process only that many records
-   */
-  private int limit;
-
-  /**
-   * below to store all the information required for aggregation during query
-   * execution
-   */
-  private AggregatorInfo aggregatorInfo;
-
-  /**
-   * this will be used to get the first tentative block from which query
-   * execution start, this will be useful in case of filter query to get the
-   * start block based on filter values
-   */
-  private IndexKey startKey;
-
-  /**
-   * this will be used to get the last tentative block till which scanning
-   * will be done, this will be useful in case of filter query to get the last
-   * block based on filter values
-   */
-  private IndexKey endKey;
-
-  /**
-   * masked byte for block which will be used to unpack the fixed length key,
-   * this will be used for updating the older block key with new block key
-   * generator
-   */
-  private int[] maskedByteForBlock;
-
-  /**
-   * flag to check whether query is detail query or aggregation query
-   */
-  private boolean isDetailQuery;
-
-  /**
-   * total number of dimension in block
-   */
-  private int totalNumberDimensionBlock;
-
-  /**
-   * total number of measure in block
-   */
-  private int totalNumberOfMeasureBlock;
-
-  /**
-   * will be used to read the dimension block from file
-   */
-  private int[] allSelectedDimensionBlocksIndexes;
-
-  /**
-   * will be used to read the measure block from file
-   */
-  private int[] allSelectedMeasureBlocksIndexes;
-
-  /**
-   * this will be used to update the older block fixed length keys with the
-   * new block fixed length key
-   */
-  private KeyStructureInfo keyStructureInfo;
-
-  /**
-   * below will be used to sort the data based
-   */
-  private SortInfo sortInfo;
-
-  /**
-   * first block from which query execution will start
-   */
-  private DataRefNode firstDataBlock;
-
-  /**
-   * number of block to be scanned in the query
-   */
-  private long numberOfBlockToScan;
-
-  /**
-   * key size of the fixed length dimension column
-   */
-  private int fixedLengthKeySize;
-
-  /**
-   * dictionary column block indexes based on query
-   */
-  private int[] dictionaryColumnBlockIndex;
-  /**
-   * no dictionary column block indexes in based on the query order
-   */
-  private int[] noDictionaryBlockIndexes;
-
-  /**
-   * key generator used for generating the table block fixed length key
-   */
-  private KeyGenerator blockKeyGenerator;
-
-  /**
-   * each column value size
-   */
-  private int[] eachColumnValueSize;
-
-  /**
-   * partition number
-   */
-  private String partitionId;
-
-  /**
-   * column group block index in file to key structure info mapping
-   */
-  private Map<Integer, KeyStructureInfo> columnGroupToKeyStructureInfo;
-
-  /**
-   * mapping of dictionary dimension to its dictionary mapping which will be
-   * used to get the actual data from dictionary for aggregation, sorting
-   */
-  private Map<String, Dictionary> columnIdToDcitionaryMapping;
-
-  /**
-   * filter tree to execute the filter
-   */
-  private FilterExecuter filterExecuterTree;
-
-  /**
-   * fileType
-   */
-  private FileType fileType;
-
-  /**
-   * whether it needs only raw byte records with out aggregation.
-   */
-  private boolean isRawRecordDetailQuery;
-
-  /**
-   * whether dimensions exist in query.
-   */
-  private boolean isDimensionsExistInQuery;
-
-  /**
-   * @return the tableBlock
-   */
-  public AbstractIndex getDataBlock() {
-    return blockIndex;
-  }
-
-  /**
-   * @param blockIndex the tableBlock to set
-   */
-  public void setDataBlock(AbstractIndex blockIndex) {
-    this.blockIndex = blockIndex;
-  }
-
-  /**
-   * @return the isFixedKeyUpdateRequired
-   */
-  public boolean isFixedKeyUpdateRequired() {
-    return isFixedKeyUpdateRequired;
-  }
-
-  /**
-   * @param isFixedKeyUpdateRequired the isFixedKeyUpdateRequired to set
-   */
-  public void setFixedKeyUpdateRequired(boolean isFixedKeyUpdateRequired) {
-    this.isFixedKeyUpdateRequired = isFixedKeyUpdateRequired;
-  }
-
-  /**
-   * @return the isFileBasedQuery
-   */
-  public boolean isFileBasedQuery() {
-    return isFileBasedQuery;
-  }
-
-  /**
-   * @param isFileBasedQuery the isFileBasedQuery to set
-   */
-  public void setFileBasedQuery(boolean isFileBasedQuery) {
-    this.isFileBasedQuery = isFileBasedQuery;
-  }
-
-  /**
-   * @return the queryId
-   */
-  public String getQueryId() {
-    return queryId;
-  }
-
-  /**
-   * @param queryId the queryId to set
-   */
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  /**
-   * @return the limit
-   */
-  public int getLimit() {
-    return limit;
-  }
-
-  /**
-   * @param limit the limit to set
-   */
-  public void setLimit(int limit) {
-    this.limit = limit;
-  }
-
-  /**
-   * @return the aggregatorInfos
-   */
-  public AggregatorInfo getAggregatorInfo() {
-    return aggregatorInfo;
-  }
-
-  /**
-   * @param aggregatorInfo the aggregatorInfos to set
-   */
-  public void setAggregatorInfo(AggregatorInfo aggregatorInfo) {
-    this.aggregatorInfo = aggregatorInfo;
-  }
-
-  /**
-   * @return the startKey
-   */
-  public IndexKey getStartKey() {
-    return startKey;
-  }
-
-  /**
-   * @param startKey the startKey to set
-   */
-  public void setStartKey(IndexKey startKey) {
-    this.startKey = startKey;
-  }
-
-  /**
-   * @return the endKey
-   */
-  public IndexKey getEndKey() {
-    return endKey;
-  }
-
-  /**
-   * @param endKey the endKey to set
-   */
-  public void setEndKey(IndexKey endKey) {
-    this.endKey = endKey;
-  }
-
-  /**
-   * @return the maskedByteForBlock
-   */
-  public int[] getMaskedByteForBlock() {
-    return maskedByteForBlock;
-  }
-
-  /**
-   * @param maskedByteForBlock the maskedByteForBlock to set
-   */
-  public void setMaskedByteForBlock(int[] maskedByteForBlock) {
-    this.maskedByteForBlock = maskedByteForBlock;
-  }
-
-  /**
-   * @return the isDetailQuery
-   */
-  public boolean isDetailQuery() {
-    return isDetailQuery;
-  }
-
-  /**
-   * @param isDetailQuery the isDetailQuery to set
-   */
-  public void setDetailQuery(boolean isDetailQuery) {
-    this.isDetailQuery = isDetailQuery;
-  }
-
-  /**
-   * @return the totalNumberDimensionBlock
-   */
-  public int getTotalNumberDimensionBlock() {
-    return totalNumberDimensionBlock;
-  }
-
-  /**
-   * @param totalNumberDimensionBlock the totalNumberDimensionBlock to set
-   */
-  public void setTotalNumberDimensionBlock(int totalNumberDimensionBlock) {
-    this.totalNumberDimensionBlock = totalNumberDimensionBlock;
-  }
-
-  /**
-   * @return the totalNumberOfMeasureBlock
-   */
-  public int getTotalNumberOfMeasureBlock() {
-    return totalNumberOfMeasureBlock;
-  }
-
-  /**
-   * @param totalNumberOfMeasureBlock the totalNumberOfMeasureBlock to set
-   */
-  public void setTotalNumberOfMeasureBlock(int totalNumberOfMeasureBlock) {
-    this.totalNumberOfMeasureBlock = totalNumberOfMeasureBlock;
-  }
-
-  /**
-   * @return the allSelectedDimensionBlocksIndexes
-   */
-  public int[] getAllSelectedDimensionBlocksIndexes() {
-    return allSelectedDimensionBlocksIndexes;
-  }
-
-  /**
-   * @param allSelectedDimensionBlocksIndexes the allSelectedDimensionBlocksIndexes to set
-   */
-  public void setAllSelectedDimensionBlocksIndexes(int[] allSelectedDimensionBlocksIndexes) {
-    this.allSelectedDimensionBlocksIndexes = allSelectedDimensionBlocksIndexes;
-  }
-
-  /**
-   * @return the allSelectedMeasureBlocksIndexes
-   */
-  public int[] getAllSelectedMeasureBlocksIndexes() {
-    return allSelectedMeasureBlocksIndexes;
-  }
-
-  /**
-   * @param allSelectedMeasureBlocksIndexes the allSelectedMeasureBlocksIndexes to set
-   */
-  public void setAllSelectedMeasureBlocksIndexes(int[] allSelectedMeasureBlocksIndexes) {
-    this.allSelectedMeasureBlocksIndexes = allSelectedMeasureBlocksIndexes;
-  }
-
-  /**
-   * @return the restructureInfos
-   */
-  public KeyStructureInfo getKeyStructureInfo() {
-    return keyStructureInfo;
-  }
-
-  /**
-   * @param keyStructureInfo the restructureInfos to set
-   */
-  public void setKeyStructureInfo(KeyStructureInfo keyStructureInfo) {
-    this.keyStructureInfo = keyStructureInfo;
-  }
-
-  /**
-   * @return the sortInfos
-   */
-  public SortInfo getSortInfo() {
-    return sortInfo;
-  }
-
-  /**
-   * @param sortInfo the sortInfos to set
-   */
-  public void setSortInfo(SortInfo sortInfo) {
-    this.sortInfo = sortInfo;
-  }
-
-  /**
-   * @return the firstDataBlock
-   */
-  public DataRefNode getFirstDataBlock() {
-    return firstDataBlock;
-  }
-
-  /**
-   * @param firstDataBlock the firstDataBlock to set
-   */
-  public void setFirstDataBlock(DataRefNode firstDataBlock) {
-    this.firstDataBlock = firstDataBlock;
-  }
-
-  /**
-   * @return the numberOfBlockToScan
-   */
-  public long getNumberOfBlockToScan() {
-    return numberOfBlockToScan;
-  }
-
-  /**
-   * @param numberOfBlockToScan the numberOfBlockToScan to set
-   */
-  public void setNumberOfBlockToScan(long numberOfBlockToScan) {
-    this.numberOfBlockToScan = numberOfBlockToScan;
-  }
-
-  /**
-   * @return the fixedLengthKeySize
-   */
-  public int getFixedLengthKeySize() {
-    return fixedLengthKeySize;
-  }
-
-  /**
-   * @param fixedLengthKeySize the fixedLengthKeySize to set
-   */
-  public void setFixedLengthKeySize(int fixedLengthKeySize) {
-    this.fixedLengthKeySize = fixedLengthKeySize;
-  }
-
-  /**
-   * @return the filterEvaluatorTree
-   */
-  public FilterExecuter getFilterExecuterTree() {
-    return filterExecuterTree;
-  }
-
-  /**
-   * @param filterExecuterTree the filterEvaluatorTree to set
-   */
-  public void setFilterExecuterTree(FilterExecuter filterExecuterTree) {
-    this.filterExecuterTree = filterExecuterTree;
-  }
-
-  /**
-   * @return the tableBlockKeyGenerator
-   */
-  public KeyGenerator getBlockKeyGenerator() {
-    return blockKeyGenerator;
-  }
-
-  /**
-   * @param tableBlockKeyGenerator the tableBlockKeyGenerator to set
-   */
-  public void setBlockKeyGenerator(KeyGenerator tableBlockKeyGenerator) {
-    this.blockKeyGenerator = tableBlockKeyGenerator;
-  }
-
-  /**
-   * @return the eachColumnValueSize
-   */
-  public int[] getEachColumnValueSize() {
-    return eachColumnValueSize;
-  }
-
-  /**
-   * @param eachColumnValueSize the eachColumnValueSize to set
-   */
-  public void setEachColumnValueSize(int[] eachColumnValueSize) {
-    this.eachColumnValueSize = eachColumnValueSize;
-  }
-
-  /**
-   * @return the partitionId
-   */
-  public String getPartitionId() {
-    return partitionId;
-  }
-
-  /**
-   * @param partitionId the partitionId to set
-   */
-  public void setPartitionId(String partitionId) {
-    this.partitionId = partitionId;
-  }
-
-  /**
-   * @return the dictionaryColumnBlockIndex
-   */
-  public int[] getDictionaryColumnBlockIndex() {
-    return dictionaryColumnBlockIndex;
-  }
-
-  /**
-   * @param dictionaryColumnBlockIndex the dictionaryColumnBlockIndex to set
-   */
-  public void setDictionaryColumnBlockIndex(int[] dictionaryColumnBlockIndex) {
-    this.dictionaryColumnBlockIndex = dictionaryColumnBlockIndex;
-  }
-
-  /**
-   * @return the noDictionaryBlockIndexes
-   */
-  public int[] getNoDictionaryBlockIndexes() {
-    return noDictionaryBlockIndexes;
-  }
-
-  /**
-   * @param noDictionaryBlockIndexes the noDictionaryBlockIndexes to set
-   */
-  public void setNoDictionaryBlockIndexes(int[] noDictionaryBlockIndexes) {
-    this.noDictionaryBlockIndexes = noDictionaryBlockIndexes;
-  }
-
-  /**
-   * @return the columnGroupToKeyStructureInfo
-   */
-  public Map<Integer, KeyStructureInfo> getColumnGroupToKeyStructureInfo() {
-    return columnGroupToKeyStructureInfo;
-  }
-
-  /**
-   * @param columnGroupToKeyStructureInfo the columnGroupToKeyStructureInfo to set
-   */
-  public void setColumnGroupToKeyStructureInfo(
-      Map<Integer, KeyStructureInfo> columnGroupToKeyStructureInfo) {
-    this.columnGroupToKeyStructureInfo = columnGroupToKeyStructureInfo;
-  }
-
-  /**
-   * @return the columnIdToDcitionaryMapping
-   */
-  public Map<String, Dictionary> getColumnIdToDcitionaryMapping() {
-    return columnIdToDcitionaryMapping;
-  }
-
-  /**
-   * @param columnIdToDcitionaryMapping the columnIdToDcitionaryMapping to set
-   */
-  public void setColumnIdToDcitionaryMapping(Map<String, Dictionary> columnIdToDcitionaryMapping) {
-    this.columnIdToDcitionaryMapping = columnIdToDcitionaryMapping;
-  }
-
-  /**
-   * @return the fileType
-   */
-  public FileType getFileType() {
-    return fileType;
-  }
-
-  /**
-   * @param fileType the fileType to set
-   */
-  public void setFileType(FileType fileType) {
-    this.fileType = fileType;
-  }
-
-  public boolean isRawRecordDetailQuery() {
-    return isRawRecordDetailQuery;
-  }
-
-  public void setRawRecordDetailQuery(boolean rawRecordDetailQuery) {
-    isRawRecordDetailQuery = rawRecordDetailQuery;
-  }
-
-  public boolean isDimensionsExistInQuery() {
-    return isDimensionsExistInQuery;
-  }
-
-  public void setDimensionsExistInQuery(boolean dimensionsExistInQuery) {
-    isDimensionsExistInQuery = dimensionsExistInQuery;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java
deleted file mode 100644
index 75e65a2..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/KeyStructureInfo.java
+++ /dev/null
@@ -1,125 +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.carbondata.query.carbon.executor.infos;
-
-import org.carbondata.core.keygenerator.KeyGenerator;
-
-/**
- * Below class will store the structure of the key
- * used during query execution
- */
-public class KeyStructureInfo {
-
-  /**
-   * it's actually a latest key generator
-   * last table block as this key generator will be used to
-   * to update the mdkey of the older slice with the new slice
-   */
-  private KeyGenerator keyGenerator;
-
-  /**
-   * mask bytes ranges for the query
-   */
-  private int[] maskByteRanges;
-
-  /**
-   * masked bytes of the query
-   */
-  private int[] maskedBytes;
-
-  /**
-   * max key for query execution
-   */
-  private byte[] maxKey;
-
-  /**
-   * mdkey start index of block
-   */
-  private int blockMdKeyStartOffset;
-
-  /**
-   * @return the keyGenerator
-   */
-  public KeyGenerator getKeyGenerator() {
-    return keyGenerator;
-  }
-
-  /**
-   * @param keyGenerator the keyGenerator to set
-   */
-  public void setKeyGenerator(KeyGenerator keyGenerator) {
-    this.keyGenerator = keyGenerator;
-  }
-
-  /**
-   * @return the maskByteRanges
-   */
-  public int[] getMaskByteRanges() {
-    return maskByteRanges;
-  }
-
-  /**
-   * @param maskByteRanges the maskByteRanges to set
-   */
-  public void setMaskByteRanges(int[] maskByteRanges) {
-    this.maskByteRanges = maskByteRanges;
-  }
-
-  /**
-   * @return the maskedBytes
-   */
-  public int[] getMaskedBytes() {
-    return maskedBytes;
-  }
-
-  /**
-   * @param maskedBytes the maskedBytes to set
-   */
-  public void setMaskedBytes(int[] maskedBytes) {
-    this.maskedBytes = maskedBytes;
-  }
-
-  /**
-   * @return the maxKey
-   */
-  public byte[] getMaxKey() {
-    return maxKey;
-  }
-
-  /**
-   * @param maxKey the maxKey to set
-   */
-  public void setMaxKey(byte[] maxKey) {
-    this.maxKey = maxKey;
-  }
-
-  /**
-   * @param startOffset
-   */
-  public void setBlockMdKeyStartOffset(int startOffset) {
-    this.blockMdKeyStartOffset = startOffset;
-  }
-
-  /**
-   * @return
-   */
-  public int getBlockMdKeyStartOffset() {
-    return this.blockMdKeyStartOffset;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/infos/SortInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/SortInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/SortInfo.java
deleted file mode 100644
index 9ab2e25..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/SortInfo.java
+++ /dev/null
@@ -1,125 +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.carbondata.query.carbon.executor.infos;
-
-import java.util.List;
-
-import org.carbondata.query.carbon.model.QueryDimension;
-
-/**
- * Below class holds the order by information about the query
- */
-public class SortInfo {
-
-  /**
-   * sorting order of a dimension
-   */
-  private byte[] dimensionSortOrder;
-
-  /**
-   * byte range of each dimension present in the order by
-   */
-  private int[][] maskedByteRangeForSorting;
-
-  /**
-   * dimension indexes which is used in order bye
-   */
-  private byte[] sortDimensionIndex;
-
-  /**
-   * mask key of each dimension
-   * this will be used to sort the dimension
-   */
-  private byte[][] dimensionMaskKeyForSorting;
-
-  /**
-   * sortDimension
-   */
-  private List<QueryDimension> sortDimension;
-
-  /**
-   * @return the dimensionSortOrder
-   */
-  public byte[] getDimensionSortOrder() {
-    return dimensionSortOrder;
-  }
-
-  /**
-   * @param dimensionSortOrder the dimensionSortOrder to set
-   */
-  public void setDimensionSortOrder(byte[] dimensionSortOrder) {
-    this.dimensionSortOrder = dimensionSortOrder;
-  }
-
-  /**
-   * @return the maskedByteRangeForSorting
-   */
-  public int[][] getMaskedByteRangeForSorting() {
-    return maskedByteRangeForSorting;
-  }
-
-  /**
-   * @param maskedByteRangeForSorting the maskedByteRangeForSorting to set
-   */
-  public void setMaskedByteRangeForSorting(int[][] maskedByteRangeForSorting) {
-    this.maskedByteRangeForSorting = maskedByteRangeForSorting;
-  }
-
-  /**
-   * @return the sortDimensionIndex
-   */
-  public byte[] getSortDimensionIndex() {
-    return sortDimensionIndex;
-  }
-
-  /**
-   * @param sortDimensionIndex the sortDimensionIndex to set
-   */
-  public void setSortDimensionIndex(byte[] sortDimensionIndex) {
-    this.sortDimensionIndex = sortDimensionIndex;
-  }
-
-  /**
-   * @return the dimensionMaskKeyForSorting
-   */
-  public byte[][] getDimensionMaskKeyForSorting() {
-    return dimensionMaskKeyForSorting;
-  }
-
-  /**
-   * @param dimensionMaskKeyForSorting the dimensionMaskKeyForSorting to set
-   */
-  public void setDimensionMaskKeyForSorting(byte[][] dimensionMaskKeyForSorting) {
-    this.dimensionMaskKeyForSorting = dimensionMaskKeyForSorting;
-  }
-
-  /**
-   * @return the sortDimension
-   */
-  public List<QueryDimension> getSortDimension() {
-    return sortDimension;
-  }
-
-  /**
-   * @param sortDimension the sortDimension to set
-   */
-  public void setSortDimension(List<QueryDimension> sortDimension) {
-    this.sortDimension = sortDimension;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java
deleted file mode 100644
index 089ee82..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java
+++ /dev/null
@@ -1,47 +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.carbondata.query.carbon.executor.internal;
-
-import java.util.List;
-
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Query executor interface which will be used to execute the
- * query
- */
-public interface InternalQueryExecutor {
-
-  /**
-   * Method will be used to execute the query
-   *
-   * @param blockExecutionInfos    list of block execution info.
-   *                               execution info will have all the parameters required for
-   *                               query execution
-   * @param blockIndexToBeExecuted index of the block to be executed
-   * @return result iterator
-   * @throws QueryExecutionException
-   */
-  CarbonIterator<Result> executeQuery(List<BlockExecutionInfo> blockExecutionInfos,
-      int[] blockIndexToBeExecuted, FileHolder fileReader) throws QueryExecutionException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
deleted file mode 100644
index 516230d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
+++ /dev/null
@@ -1,718 +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.carbondata.query.carbon.executor.util;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Iterator;
-import java.util.List;
-import java.util.Map;
-import java.util.Map.Entry;
-import java.util.Set;
-import java.util.TreeSet;
-
-import org.carbondata.core.cache.Cache;
-import org.carbondata.core.cache.CacheProvider;
-import org.carbondata.core.cache.CacheType;
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.CarbonTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.metadata.CarbonMetadata;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.core.util.CarbonUtilException;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-
-import org.apache.commons.lang3.ArrayUtils;
-
-/**
- * Utility class for query execution
- */
-public class QueryUtil {
-
-  /**
-   * Below method will be used to get the masked byte range based on the query
-   * dimension. It will give the range in the mdkey. This will be used to get
-   * the actual key array from masked mdkey
-   *
-   * @param queryDimensions query dimension selected in query
-   * @param keyGenerator    key generator
-   * @return masked key
-   */
-  public static int[] getMaskedByteRange(List<QueryDimension> queryDimensions,
-      KeyGenerator keyGenerator) {
-    Set<Integer> byteRangeSet = new TreeSet<Integer>();
-    int[] byteRange = null;
-    for (int i = 0; i < queryDimensions.size(); i++) {
-
-      // as no dictionary column and complex type columns
-      // are not selected in the mdkey
-      // so we will not select the those dimension for calculating the
-      // range
-      if (queryDimensions.get(i).getDimension().getKeyOrdinal() == -1) {
-        continue;
-      }
-      // get the offset of the dimension in the mdkey
-      byteRange =
-          keyGenerator.getKeyByteOffsets(queryDimensions.get(i).getDimension().getKeyOrdinal());
-      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
-        byteRangeSet.add(j);
-      }
-    }
-    int[] maksedByteRange = new int[byteRangeSet.size()];
-    int index = 0;
-    Iterator<Integer> iterator = byteRangeSet.iterator();
-    // add the masked byte range
-    while (iterator.hasNext()) {
-      maksedByteRange[index++] = iterator.next();
-    }
-    return maksedByteRange;
-  }
-
-  public static int[] getMaskedByteRangeBasedOrdinal(List<Integer> ordinals,
-      KeyGenerator keyGenerator) {
-    Set<Integer> byteRangeSet = new TreeSet<Integer>();
-    int[] byteRange = null;
-    for (int i = 0; i < ordinals.size(); i++) {
-
-      // get the offset of the dimension in the mdkey
-      byteRange = keyGenerator.getKeyByteOffsets(ordinals.get(i));
-      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
-        byteRangeSet.add(j);
-      }
-    }
-    int[] maksedByteRange = new int[byteRangeSet.size()];
-    int index = 0;
-    Iterator<Integer> iterator = byteRangeSet.iterator();
-    // add the masked byte range
-    while (iterator.hasNext()) {
-      maksedByteRange[index++] = iterator.next();
-    }
-    return maksedByteRange;
-  }
-
-  /**
-   * Below method will return the max key based on the dimension ordinal
-   *
-   * @param keyOrdinalList
-   * @param generator
-   * @return
-   * @throws KeyGenException
-   */
-  public static byte[] getMaxKeyBasedOnOrinal(List<Integer> keyOrdinalList, KeyGenerator generator)
-      throws KeyGenException {
-    long[] max = new long[generator.getDimCount()];
-    Arrays.fill(max, 0L);
-
-    for (int i = 0; i < keyOrdinalList.size(); i++) {
-      // adding for dimension which is selected in query
-      max[keyOrdinalList.get(i)] = Long.MAX_VALUE;
-    }
-    return generator.generateKey(max);
-  }
-
-  /**
-   * To get the max key based on dimensions. i.e. all other dimensions will be
-   * set to 0 bits and the required query dimension will be masked with all
-   * LONG.MAX so that we can mask key and then compare while aggregating This
-   * can be useful during filter query when only few dimensions were selected
-   * out of row group
-   *
-   * @param queryDimensions dimension selected in query
-   * @param generator       key generator
-   * @return max key for dimension
-   * @throws KeyGenException if any problem while generating the key
-   */
-  public static byte[] getMaxKeyBasedOnDimensions(List<QueryDimension> queryDimensions,
-      KeyGenerator generator) throws KeyGenException {
-    long[] max = new long[generator.getDimCount()];
-    Arrays.fill(max, 0L);
-
-    for (int i = 0; i < queryDimensions.size(); i++) {
-      // as no dictionary column and complex type columns
-      // are not selected in the mdkey
-      // so we will not select the those dimension for calculating the
-      // range
-      if (queryDimensions.get(i).getDimension().getKeyOrdinal() == -1) {
-        continue;
-      }
-      // adding for dimension which is selected in query
-      max[queryDimensions.get(i).getDimension().getKeyOrdinal()] = Long.MAX_VALUE;
-    }
-
-    return generator.generateKey(max);
-  }
-
-  /**
-   * Below method will be used to get the masked key for query
-   *
-   * @param keySize         size of the masked key
-   * @param maskedKeyRanges masked byte range
-   * @return masked bytes
-   */
-  public static int[] getMaskedByte(int keySize, int[] maskedKeyRanges) {
-    int[] maskedKey = new int[keySize];
-    // all the non selected dimension will be filled with -1
-    Arrays.fill(maskedKey, -1);
-    for (int i = 0; i < maskedKeyRanges.length; i++) {
-      maskedKey[maskedKeyRanges[i]] = i;
-    }
-    return maskedKey;
-  }
-
-  /**
-   * Below method will be used to get the dimension block index in file based
-   * on query dimension
-   *
-   * @param queryDimensions                query dimension
-   * @param dimensionOrdinalToBlockMapping mapping of dimension block in file to query dimension
-   * @return block index of file
-   */
-  public static int[] getDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
-      Map<Integer, Integer> dimensionOrdinalToBlockMapping,
-      List<CarbonDimension> customAggregationDimension) {
-    // using set as in row group columns will point to same block
-    Set<Integer> dimensionBlockIndex = new HashSet<Integer>();
-    for (int i = 0; i < queryDimensions.size(); i++) {
-      dimensionBlockIndex.add(
-          dimensionOrdinalToBlockMapping.get(queryDimensions.get(i).getDimension().getOrdinal()));
-    }
-    for (int i = 0; i < customAggregationDimension.size(); i++) {
-      dimensionBlockIndex
-          .add(dimensionOrdinalToBlockMapping.get(customAggregationDimension.get(i).getOrdinal()));
-    }
-    return ArrayUtils
-        .toPrimitive(dimensionBlockIndex.toArray(new Integer[dimensionBlockIndex.size()]));
-  }
-
-  /**
-   * Below method will be used to get the dictionary mapping for all the
-   * dictionary encoded dimension present in the query
-   *
-   * @param queryDimensions            query dimension present in the query this will be used to
-   *                                   convert the result from surrogate key to actual data
-   * @param absoluteTableIdentifier    absolute table identifier
-   * @return dimension unique id to its dictionary map
-   * @throws QueryExecutionException
-   */
-  public static Map<String, Dictionary> getDimensionDictionaryDetail(
-      List<QueryDimension> queryDimensions,
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws QueryExecutionException {
-    // to store dimension unique column id list, this is required as
-    // dimension can be present in
-    // query dimension, as well as some aggregation function will be applied
-    // in the same dimension
-    // so we need to get only one instance of dictionary
-    // direct dictionary skip is done only for the dictionary lookup
-    Set<String> dictionaryDimensionFromQuery = new HashSet<String>();
-    for (int i = 0; i < queryDimensions.size(); i++) {
-      List<Encoding> encodingList = queryDimensions.get(i).getDimension().getEncoder();
-      if (CarbonUtil.hasEncoding(encodingList, Encoding.DICTIONARY) && !CarbonUtil
-          .hasEncoding(encodingList, Encoding.DIRECT_DICTIONARY)) {
-        dictionaryDimensionFromQuery.add(queryDimensions.get(i).getDimension().getColumnId());
-      }
-    }
-    // converting to list as api exposed needed list which i think
-    // is not correct
-    List<String> dictionaryColumnIdList =
-        new ArrayList<String>(dictionaryDimensionFromQuery.size());
-    dictionaryColumnIdList.addAll(dictionaryDimensionFromQuery);
-    return getDictionaryMap(dictionaryColumnIdList, absoluteTableIdentifier);
-  }
-
-  /**
-   * Below method will be used to get the column id to its dictionary mapping
-   *
-   * @param dictionaryColumnIdList  dictionary column list
-   * @param absoluteTableIdentifier absolute table identifier
-   * @return dictionary mapping
-   * @throws QueryExecutionException
-   */
-  private static Map<String, Dictionary> getDictionaryMap(List<String> dictionaryColumnIdList,
-      AbsoluteTableIdentifier absoluteTableIdentifier) throws QueryExecutionException {
-    // this for dictionary unique identifier
-    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
-        getDictionaryColumnUniqueIdentifierList(dictionaryColumnIdList,
-            absoluteTableIdentifier.getCarbonTableIdentifier());
-    CacheProvider cacheProvider = CacheProvider.getInstance();
-    Cache forwardDictionaryCache = cacheProvider
-        .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
-    List<Dictionary> columnDictionaryList = null;
-    try {
-      columnDictionaryList = forwardDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
-    } catch (CarbonUtilException e) {
-      throw new QueryExecutionException(e);
-    }
-    Map<String, Dictionary> columnDictionaryMap = new HashMap<>(columnDictionaryList.size());
-    for (int i = 0; i < dictionaryColumnUniqueIdentifiers.size(); i++) {
-      // TODO: null check for column dictionary, if cache size is less it
-      // might return null here, in that case throw exception
-      columnDictionaryMap.put(dictionaryColumnIdList.get(i), columnDictionaryList.get(i));
-    }
-    return columnDictionaryMap;
-  }
-
-  /**
-   * Below method will be used to get the dictionary column unique identifier
-   *
-   * @param dictionaryColumnIdList dictionary
-   * @param carbonTableIdentifier
-   * @return
-   */
-  private static List<DictionaryColumnUniqueIdentifier> getDictionaryColumnUniqueIdentifierList(
-      List<String> dictionaryColumnIdList, CarbonTableIdentifier carbonTableIdentifier) {
-    CarbonTable carbonTable =
-        CarbonMetadata.getInstance().getCarbonTable(carbonTableIdentifier.getTableUniqueName());
-    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
-        new ArrayList<>(dictionaryColumnIdList.size());
-    for (String columnIdentifier : dictionaryColumnIdList) {
-      CarbonDimension dimension = CarbonMetadata.getInstance()
-          .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnIdentifier);
-      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-          new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
-              dimension.getDataType());
-      dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
-    }
-    return dictionaryColumnUniqueIdentifiers;
-  }
-
-  /**
-   * Below method will used to get the method will be used to get the measure
-   * block indexes to be read from the file
-   *
-   * @param queryMeasures              query measure
-   * @param expressionMeasure          measure present in the expression
-   * @param ordinalToBlockIndexMapping measure ordinal to block mapping
-   * @return block indexes
-   */
-  public static int[] getMeasureBlockIndexes(List<QueryMeasure> queryMeasures,
-      List<CarbonMeasure> expressionMeasure, Map<Integer, Integer> ordinalToBlockIndexMapping) {
-    Set<Integer> measureBlockIndex = new HashSet<Integer>();
-    for (int i = 0; i < queryMeasures.size(); i++) {
-      measureBlockIndex
-          .add(ordinalToBlockIndexMapping.get(queryMeasures.get(i).getMeasure().getOrdinal()));
-    }
-    for (int i = 0; i < expressionMeasure.size(); i++) {
-      measureBlockIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal()));
-    }
-    return ArrayUtils.toPrimitive(measureBlockIndex.toArray(new Integer[measureBlockIndex.size()]));
-  }
-
-  /**
-   * Below method will be used to get the masked byte range for dimension
-   * which is present in order by
-   *
-   * @param orderByDimensions order by dimension
-   * @param generator         key generator
-   * @param maskedRanges      masked byte range for dimension
-   * @return range of masked byte for order by dimension
-   */
-  public static int[][] getMaskedByteRangeForSorting(List<QueryDimension> orderByDimensions,
-      KeyGenerator generator, int[] maskedRanges) {
-    int[][] dimensionCompareIndex = new int[orderByDimensions.size()][];
-    int index = 0;
-    for (int i = 0; i < dimensionCompareIndex.length; i++) {
-      Set<Integer> integers = new TreeSet<Integer>();
-      if (!orderByDimensions.get(i).getDimension().getEncoder().contains(Encoding.DICTIONARY)
-          || orderByDimensions.get(i).getDimension().numberOfChild() > 0) {
-        continue;
-      }
-      int[] range =
-          generator.getKeyByteOffsets(orderByDimensions.get(i).getDimension().getKeyOrdinal());
-      for (int j = range[0]; j <= range[1]; j++) {
-        integers.add(j);
-      }
-      dimensionCompareIndex[index] = new int[integers.size()];
-      int j = 0;
-      for (Iterator<Integer> iterator = integers.iterator(); iterator.hasNext(); ) {
-        Integer integer = (Integer) iterator.next();
-        dimensionCompareIndex[index][j++] = integer.intValue();
-      }
-      index++;
-    }
-    for (int i = 0; i < dimensionCompareIndex.length; i++) {
-      if (null == dimensionCompareIndex[i]) {
-        continue;
-      }
-      int[] range = dimensionCompareIndex[i];
-      if (null != range) {
-        for (int j = 0; j < range.length; j++) {
-          for (int k = 0; k < maskedRanges.length; k++) {
-            if (range[j] == maskedRanges[k]) {
-              range[j] = k;
-              break;
-            }
-          }
-        }
-      }
-
-    }
-    return dimensionCompareIndex;
-  }
-
-  /**
-   * Below method will be used to get the masked key for sorting
-   *
-   * @param orderDimensions           query dimension
-   * @param generator                 key generator
-   * @param maskedByteRangeForSorting masked byte range for sorting
-   * @param maskedRanges              masked range
-   * @return masked byte range
-   * @throws QueryExecutionException
-   */
-  public static byte[][] getMaksedKeyForSorting(List<QueryDimension> orderDimensions,
-      KeyGenerator generator, int[][] maskedByteRangeForSorting, int[] maskedRanges)
-      throws QueryExecutionException {
-    byte[][] maskedKey = new byte[orderDimensions.size()][];
-    byte[] mdKey = null;
-    long[] key = null;
-    byte[] maskedMdKey = null;
-    try {
-      if (null != maskedByteRangeForSorting) {
-        for (int i = 0; i < maskedByteRangeForSorting.length; i++) {
-          if (null == maskedByteRangeForSorting[i]) {
-            continue;
-          }
-          key = new long[generator.getDimCount()];
-          maskedKey[i] = new byte[maskedByteRangeForSorting[i].length];
-          key[orderDimensions.get(i).getDimension().getKeyOrdinal()] = Long.MAX_VALUE;
-          mdKey = generator.generateKey(key);
-          maskedMdKey = new byte[maskedRanges.length];
-          for (int k = 0; k < maskedMdKey.length; k++) { // CHECKSTYLE:OFF
-            // Approval
-            // No:Approval-V1R2C10_001
-            maskedMdKey[k] = mdKey[maskedRanges[k]];
-          }
-          for (int j = 0; j < maskedByteRangeForSorting[i].length; j++) {
-            maskedKey[i][j] = maskedMdKey[maskedByteRangeForSorting[i][j]];
-          }// CHECKSTYLE:ON
-
-        }
-      }
-    } catch (KeyGenException e) {
-      throw new QueryExecutionException(e);
-    }
-    return maskedKey;
-  }
-
-  /**
-   * Below method will be used to get mapping whether dimension is present in
-   * order by or not
-   *
-   * @param sortedDimensions sort dimension present in order by query
-   * @param queryDimensions  query dimension
-   * @return sort dimension indexes
-   */
-  public static byte[] getSortDimensionIndexes(List<QueryDimension> sortedDimensions,
-      List<QueryDimension> queryDimensions) {
-    byte[] sortedDims = new byte[queryDimensions.size()];
-    int indexOf = 0;
-    for (int i = 0; i < sortedDims.length; i++) {
-      indexOf = sortedDimensions.indexOf(queryDimensions.get(i));
-      if (indexOf > -1) {
-        sortedDims[i] = 1;
-      }
-    }
-    return sortedDims;
-  }
-
-  /**
-   * Below method will be used to get the mapping of block index and its
-   * restructuring info
-   *
-   * @param queryDimensions   query dimension from query model
-   * @param segmentProperties segment properties
-   * @return map of block index to its restructuring info
-   * @throws KeyGenException if problem while key generation
-   */
-  public static Map<Integer, KeyStructureInfo> getColumnGroupKeyStructureInfo(
-      List<QueryDimension> queryDimensions, SegmentProperties segmentProperties)
-      throws KeyGenException {
-    Map<Integer, KeyStructureInfo> rowGroupToItsRSInfo = new HashMap<Integer, KeyStructureInfo>();
-    // get column group id and its ordinal mapping of column group
-    Map<Integer, List<Integer>> columnGroupAndItsOrdinalMappingForQuery =
-        getColumnGroupAndItsOrdinalMapping(queryDimensions);
-    KeyGenerator keyGenerator = segmentProperties.getDimensionKeyGenerator();
-
-    Iterator<Entry<Integer, List<Integer>>> iterator =
-        columnGroupAndItsOrdinalMappingForQuery.entrySet().iterator();
-    KeyStructureInfo restructureInfos = null;
-    while (iterator.hasNext()) {
-      Entry<Integer, List<Integer>> next = iterator.next();
-      restructureInfos = new KeyStructureInfo();
-      // sort the ordinal
-      List<Integer> ordinal = next.getValue();
-      Collections.sort(ordinal);
-      // get the masked byte range for column group
-      int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(ordinal, keyGenerator);
-      // max key for column group
-      byte[] maxKey = getMaxKeyBasedOnOrinal(ordinal, keyGenerator);
-      // get masked key for column group
-      int[] maksedByte = getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
-      restructureInfos.setKeyGenerator(keyGenerator);
-      restructureInfos.setMaskByteRanges(maskByteRanges);
-      restructureInfos.setMaxKey(maxKey);
-      restructureInfos.setMaskedBytes(maksedByte);
-      restructureInfos
-          .setBlockMdKeyStartOffset(getBlockMdKeyStartOffset(segmentProperties, ordinal));
-      rowGroupToItsRSInfo
-          .put(segmentProperties.getDimensionOrdinalToBlockMapping().get(ordinal.get(0)),
-              restructureInfos);
-    }
-    return rowGroupToItsRSInfo;
-  }
-
-  /**
-   * It return mdkey start index of given column group
-   * @param segmentProperties
-   * @param ordinal : column group ordinal
-   * @return
-   */
-  public static int getBlockMdKeyStartOffset(SegmentProperties segmentProperties,
-      List<Integer> ordinal) {
-    int[][] colGroups = segmentProperties.getColumnGroups();
-    int blockMdkeyStartOffset = 0;
-    for (int i = 0; i < colGroups.length; i++) {
-      if (QueryUtil.searchInArray(colGroups[i], ordinal.get(0))) {
-        break;
-      }
-      blockMdkeyStartOffset += segmentProperties.getDimensionColumnsValueSize()[i];
-    }
-    return blockMdkeyStartOffset;
-  }
-
-  /**
-   * return true if given key is found in array
-   *
-   * @param data
-   * @param key
-   * @return
-   */
-  public static boolean searchInArray(int[] data, int key) {
-    for (int i = 0; i < data.length; i++) {
-      if (key == data[i]) {
-        return true;
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Below method will be used to create a mapping of column group columns
-   * this mapping will have column group id to all the dimension ordinal
-   * present in the column group This mapping will be used during query
-   * execution, to create a mask key for the column group dimension which will
-   * be used in aggregation and filter query as column group dimension will be
-   * stored in bit level
-   */
-  private static Map<Integer, List<Integer>> getColumnGroupAndItsOrdinalMapping(
-      List<QueryDimension> origdimensions) {
-
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>(origdimensions.size());
-    dimensions.addAll(origdimensions);
-    /**
-     * sort based on column group id
-     */
-    Collections.sort(dimensions, new Comparator<QueryDimension>() {
-
-      @Override public int compare(QueryDimension o1, QueryDimension o2) {
-        return Integer
-            .compare(o1.getDimension().columnGroupId(), o2.getDimension().columnGroupId());
-      }
-    });
-    // list of row groups this will store all the row group column
-    Map<Integer, List<Integer>> columnGroupAndItsOrdinalsMapping =
-        new HashMap<Integer, List<Integer>>();
-    // to store a column group
-    List<Integer> currentColumnGroup = null;
-    // current index
-    int index = 0;
-    // previous column group to check all the column of row id has bee
-    // selected
-    int prvColumnGroupId = -1;
-    while (index < dimensions.size()) {
-      // if dimension group id is not zero and it is same as the previous
-      // column group id
-      // then we need to add ordinal of that column as it belongs to same
-      // column group
-      if (!dimensions.get(index).getDimension().isColumnar()
-          && dimensions.get(index).getDimension().columnGroupId() == prvColumnGroupId) {
-        currentColumnGroup.add(dimensions.get(index).getDimension().getOrdinal());
-      }
-
-      // if dimension is not a columnar then it is column group column
-      else if (!dimensions.get(index).getDimension().isColumnar()) {
-        currentColumnGroup = new ArrayList<Integer>();
-        columnGroupAndItsOrdinalsMapping
-            .put(dimensions.get(index).getDimension().columnGroupId(), currentColumnGroup);
-        currentColumnGroup.add(dimensions.get(index).getDimension().getOrdinal());
-      }
-      // update the row id every time,this is required to group the
-      // columns
-      // of the same row group
-      prvColumnGroupId = dimensions.get(index).getDimension().columnGroupId();
-      index++;
-    }
-    return columnGroupAndItsOrdinalsMapping;
-  }
-
-  /**
-   * Below method will be used to get masked byte
-   *
-   * @param data           actual data
-   * @param maxKey         max key
-   * @param maskByteRanges mask byte range
-   * @param byteCount
-   * @return masked byte
-   */
-  public static byte[] getMaskedKey(byte[] data, byte[] maxKey, int[] maskByteRanges,
-      int byteCount) {
-    byte[] maskedKey = new byte[byteCount];
-    int counter = 0;
-    int byteRange = 0;
-    for (int i = 0; i < byteCount; i++) {
-      byteRange = maskByteRanges[i];
-      if (byteRange != -1) {
-        maskedKey[counter++] = (byte) (data[byteRange] & maxKey[byteRange]);
-      }
-    }
-    return maskedKey;
-  }
-
-  /**
-   * Below method will be used to fill block indexes of the query dimension
-   * which will be used in creating a output row Here is method we are passing
-   * two list which store the indexes one for dictionary column other for not
-   * dictionary column. This is done for specific purpose so that in one
-   * iteration we will be able to fill both type dimension block indexes
-   *
-   * @param queryDimensions                  dimension present in the query
-   * @param columnOrdinalToBlockIndexMapping column ordinal to block index mapping
-   * @param dictionaryDimensionBlockIndex    list to store dictionary column block indexes
-   * @param noDictionaryDimensionBlockIndex  list to store no dictionary block indexes
-   */
-  public static void fillQueryDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
-      Map<Integer, Integer> columnOrdinalToBlockIndexMapping,
-      Set<Integer> dictionaryDimensionBlockIndex, List<Integer> noDictionaryDimensionBlockIndex) {
-    for (QueryDimension queryDimension : queryDimensions) {
-      if (CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)) {
-        dictionaryDimensionBlockIndex
-            .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal()));
-      } else {
-        noDictionaryDimensionBlockIndex
-            .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal()));
-      }
-    }
-  }
-
-  /**
-   * Below method will be used to resolve the query model
-   * resolve will be setting the actual dimension and measure object
-   * as from driver only column name will be passes to avoid the heavy object
-   * serialization
-   *
-   * @param queryModel query model
-   */
-  public static void resolveQueryModel(QueryModel queryModel) {
-    CarbonMetadata.getInstance().addCarbonTable(queryModel.getTable());
-    // TODO need to load the table from table identifier
-    CarbonTable carbonTable = queryModel.getTable();
-    String tableName =
-        queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName();
-    // resolve query dimension
-    for (QueryDimension queryDimension : queryModel.getQueryDimension()) {
-      queryDimension
-          .setDimension(carbonTable.getDimensionByName(tableName, queryDimension.getColumnName()));
-    }
-    // resolve sort dimension
-    for (QueryDimension sortDimension : queryModel.getSortDimension()) {
-      sortDimension
-          .setDimension(carbonTable.getDimensionByName(tableName, sortDimension.getColumnName()));
-    }
-    // resolve query measure
-    for (QueryMeasure queryMeasure : queryModel.getQueryMeasures()) {
-      // in case of count start column name will  be count * so
-      // for count start add first measure if measure is not present
-      // than add first dimension as a measure
-      if (queryMeasure.getColumnName().equals("count(*)")) {
-        if (carbonTable.getMeasureByTableName(tableName).size() > 0) {
-          queryMeasure.setMeasure(carbonTable.getMeasureByTableName(tableName).get(0));
-        } else {
-          CarbonMeasure dummyMeasure = new CarbonMeasure(
-              carbonTable.getDimensionByTableName(tableName).get(0).getColumnSchema(), 0);
-          queryMeasure.setMeasure(dummyMeasure);
-        }
-      } else {
-        queryMeasure
-            .setMeasure(carbonTable.getMeasureByName(tableName, queryMeasure.getColumnName()));
-      }
-    }
-    //TODO need to handle expression
-  }
-
-  /**
-   * Below method will be used to get the index of number type aggregator
-   *
-   * @param aggType
-   * @return index in aggregator
-   */
-  public static int[] getNumberTypeIndex(List<String> aggType) {
-    List<Integer> indexList = new ArrayList<Integer>();
-    for (int i = 0; i < aggType.size(); i++) {
-      if (CarbonCommonConstants.SUM.equals(aggType.get(i)) || CarbonCommonConstants.AVERAGE
-          .equals(aggType.get(i))) {
-        indexList.add(i);
-      }
-    }
-    return ArrayUtils.toPrimitive(indexList.toArray(new Integer[indexList.size()]));
-  }
-
-  /**
-   * below method will be used to get the actual type aggregator
-   *
-   * @param aggType
-   * @return index in aggrgetor
-   */
-  public static int[] getActualTypeIndex(List<String> aggType) {
-    List<Integer> indexList = new ArrayList<Integer>();
-    for (int i = 0; i < aggType.size(); i++) {
-      if (!CarbonCommonConstants.SUM.equals(aggType.get(i)) && !CarbonCommonConstants.AVERAGE
-          .equals(aggType.get(i))) {
-        indexList.add(i);
-      }
-    }
-    return ArrayUtils.toPrimitive(indexList.toArray(new Integer[indexList.size()]));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/util/RestructureUtil.java b/core/src/main/java/org/carbondata/query/carbon/executor/util/RestructureUtil.java
deleted file mode 100644
index 166da92..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/util/RestructureUtil.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.carbondata.query.carbon.executor.util;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbon.executor.infos.AggregatorInfo;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-
-/**
- * Utility class for restructuring
- */
-public class RestructureUtil {
-
-  /**
-   * Below method will be used to get the updated query dimension updation
-   * means, after restructuring some dimension will be not present in older
-   * table blocks in that case we need to select only those dimension out of
-   * query dimension which is present in the current table block
-   *
-   * @param queryDimensions
-   * @param tableBlockDimensions
-   * @return list of query dimension which is present in the table block
-   */
-  public static List<QueryDimension> getUpdatedQueryDimension(
-      List<QueryDimension> queryDimensions, List<CarbonDimension> tableBlockDimensions) {
-    List<QueryDimension> presentDimension =
-        new ArrayList<QueryDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // selecting only those dimension which is present in the query
-    for (QueryDimension queryDimimension : queryDimensions) {
-      for (CarbonDimension tableDimension : tableBlockDimensions) {
-        if (tableDimension.equals(queryDimimension.getDimension())) {
-          presentDimension.add(queryDimimension);
-        }
-      }
-    }
-    return presentDimension;
-  }
-
-  /**
-   * Below method is to add dimension children for complex type dimension as
-   * internally we are creating dimension column for each each complex
-   * dimension so when complex query dimension request will come in the query,
-   * we need to add its children as it is hidden from the user For example if
-   * complex dimension is of Array of String[2] so we are storing 3 dimension
-   * and when user will query for complex type i.e. array type we need to add
-   * its children and then we will read respective block and create a tuple
-   * based on all three dimension
-   *
-   * @param queryDimensions      current query dimensions
-   * @param tableBlockDimensions dimensions which is present in the table block
-   * @return updated dimension(after adding complex type children)
-   */
-  public static List<CarbonDimension> addChildrenForComplexTypeDimension(
-      List<CarbonDimension> queryDimensions, List<CarbonDimension> tableBlockDimensions) {
-    List<CarbonDimension> updatedQueryDimension = new ArrayList<CarbonDimension>();
-    int numberOfChildren = 0;
-    for (CarbonDimension queryDimension : queryDimensions) {
-      // if number of child is zero, then it is not a complex dimension
-      // so directly add it query dimension
-      if (queryDimension.numberOfChild() == 0) {
-        updatedQueryDimension.add(queryDimension);
-      }
-      // if number of child is more than 1 then add all its children
-      numberOfChildren = queryDimension.getOrdinal() + queryDimension.numberOfChild();
-      for (int j = queryDimension.getOrdinal(); j < numberOfChildren; j++) {
-        updatedQueryDimension.add(tableBlockDimensions.get(j));
-      }
-    }
-    return updatedQueryDimension;
-  }
-
-  /**
-   * Below method will be used to get the aggregator info object
-   * in this method some of the properties which will be extracted
-   * from query measure and current block measures will be set
-   *
-   * @param queryMeasures        measures present in query
-   * @param currentBlockMeasures current block measures
-   * @return aggregator info
-   */
-  public static AggregatorInfo getAggregatorInfos(List<QueryMeasure> queryMeasures,
-      List<CarbonMeasure> currentBlockMeasures) {
-    AggregatorInfo aggregatorInfos = new AggregatorInfo();
-    int numberOfMeasureInQuery = queryMeasures.size();
-    int[] measureOrdinals = new int[numberOfMeasureInQuery];
-    Object[] defaultValues = new Object[numberOfMeasureInQuery];
-    boolean[] measureExistsInCurrentBlock = new boolean[numberOfMeasureInQuery];
-    int index = 0;
-    for (QueryMeasure queryMeasure : queryMeasures) {
-      measureOrdinals[index] = queryMeasure.getMeasure().getOrdinal();
-      // if query measure exists in current dimension measures
-      // then setting measure exists is true
-      // otherwise adding a default value of a measure
-      if (currentBlockMeasures.contains(queryMeasure.getMeasure())) {
-        measureExistsInCurrentBlock[index] = true;
-      } else {
-        defaultValues[index] = queryMeasure.getMeasure().getDefaultValue();
-      }
-      index++;
-    }
-    aggregatorInfos.setDefaultValues(defaultValues);
-    aggregatorInfos.setMeasureOrdinals(measureOrdinals);
-    aggregatorInfos.setMeasureExists(measureExistsInCurrentBlock);
-    return aggregatorInfos;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java b/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
deleted file mode 100644
index 3684f4a..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
+++ /dev/null
@@ -1,243 +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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.expression.Expression;
-
-/**
- * This class contains all the logical information about the query like dimensions,measures,
- * sort order, topN etc..
- */
-public class CarbonQueryPlan implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = -9036044826928017164L;
-
-  /**
-   * Schema name , if user asks select * from datasight.employee.
-   * then datasight is the schame name.
-   * Remains null if the user does not select schema name.
-   */
-  private String schemaName;
-
-  /**
-   * Cube name .
-   * if user asks select * from datasight.employee. then employee is the cube name.
-   * It is mandatory.
-   */
-  private String cubeName;
-
-  /**
-   * List of dimensions.
-   * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name
-   * and department_name are dimensions
-   * If there is no dimensions asked in query then it would be remained as empty.
-   */
-  private List<QueryDimension> dimensions =
-      new ArrayList<QueryDimension>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  /**
-   * List of measures.
-   * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary)
-   * would be measure.
-   * If there is no dimensions asked in query then it would be remained as empty.
-   */
-  private List<QueryMeasure> measures =
-      new ArrayList<QueryMeasure>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-  /**
-   * Limit
-   */
-  private int limit = -1;
-
-  /**
-   * If it is detail query, no need to aggregate in backend
-   */
-  private boolean detailQuery;
-
-  /**
-   * expression
-   */
-  private Expression expression;
-
-  /**
-   * queryId
-   */
-  private String queryId;
-
-  /**
-   * outLocationPath
-   */
-  private String outLocationPath;
-
-  /**
-   * isCountStarQuery
-   */
-  private boolean isCountStartQuery;
-
-  private List<QueryDimension> sortedDimensions;
-
-  /**
-   * If it is raw detail query, no need to aggregate in backend. And it reurns with dictionary data
-   * with out decoding.
-   */
-  private boolean rawDetailQuery;
-
-  /**
-   * Constructor created with cube name.
-   *
-   * @param cubeName
-   */
-  public CarbonQueryPlan(String cubeName) {
-    this.cubeName = cubeName;
-  }
-
-  /**
-   * Constructor created with schema name and cube name.
-   *
-   * @param schemaName
-   * @param cubeName
-   */
-  public CarbonQueryPlan(String schemaName, String cubeName) {
-    this.cubeName = cubeName;
-    this.schemaName = schemaName;
-  }
-
-  /**
-   * @return the dimensions
-   */
-  public List<QueryDimension> getDimensions() {
-    return dimensions;
-  }
-
-  public void addDimension(QueryDimension dimension) {
-    this.dimensions.add(dimension);
-  }
-
-  /**
-   * @return the measures
-   */
-  public List<QueryMeasure> getMeasures() {
-    return measures;
-  }
-
-  public void addMeasure(QueryMeasure measure) {
-    this.measures.add(measure);
-  }
-
-  public Expression getFilterExpression() {
-    return expression;
-  }
-
-  public void setFilterExpression(Expression expression) {
-    this.expression = expression;
-  }
-
-  /**
-   * @return the schemaName
-   */
-  public String getSchemaName() {
-    return schemaName;
-  }
-
-  /**
-   * @return the cubeName
-   */
-  public String getCubeName() {
-    return cubeName;
-  }
-
-  /**
-   * @return the limit
-   */
-  public int getLimit() {
-    return limit;
-  }
-
-  /**
-   * @param limit the limit to set
-   */
-  public void setLimit(int limit) {
-    this.limit = limit;
-  }
-
-  /**
-   * @return the detailQuery
-   */
-  public boolean isDetailQuery() {
-    return detailQuery;
-  }
-
-  /**
-   * @param detailQuery the detailQuery to set
-   */
-  public void setDetailQuery(boolean detailQuery) {
-    this.detailQuery = detailQuery;
-  }
-
-  public String getQueryId() {
-    return queryId;
-  }
-
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  public String getOutLocationPath() {
-    return outLocationPath;
-  }
-
-  public void setOutLocationPath(String outLocationPath) {
-    this.outLocationPath = outLocationPath;
-  }
-
-  public boolean isCountStarQuery() {
-    return isCountStartQuery;
-  }
-
-  public void setCountStartQuery(boolean isCountStartQuery) {
-    this.isCountStartQuery = isCountStartQuery;
-  }
-
-  public List<QueryDimension> getSortedDimemsions() {
-    return sortedDimensions;
-  }
-
-  public void setSortedDimemsions(List<QueryDimension> dims) {
-    this.sortedDimensions = dims;
-  }
-
-  public boolean isRawDetailQuery() {
-    return rawDetailQuery;
-  }
-
-  public void setRawDetailQuery(boolean rawDetailQuery) {
-    this.rawDetailQuery = rawDetailQuery;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/DimensionAggregatorInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/DimensionAggregatorInfo.java b/core/src/main/java/org/carbondata/query/carbon/model/DimensionAggregatorInfo.java
deleted file mode 100644
index b6f8976..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/DimensionAggregatorInfo.java
+++ /dev/null
@@ -1,113 +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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-
-/**
- * Dimension aggregation info which will be used to
- * holds the information about dimension and aggregation applied in the dimension
- */
-public class DimensionAggregatorInfo implements Serializable {
-
-  /**
-   * serialization version
-   */
-  private static final long serialVersionUID = 4801602263271340969L;
-
-  /**
-   * name of the column in which aggregation is applied
-   */
-  private String columnName;
-
-  /**
-   * dimension in which aggregation is applied
-   */
-  private transient CarbonDimension dim;
-
-  /**
-   * list if aggregate function applied in the dimension
-   */
-  private List<String> aggList;
-
-  /**
-   * order of the aggregate function in which output of aggregation will be
-   * send from executor to driver. Integer represents the order of the output
-   */
-  private List<Integer> orderOfAggregation;
-
-  /**
-   * @return the columnName
-   */
-  public String getColumnName() {
-    return columnName;
-  }
-
-  /**
-   * @param columnName the columnName to set
-   */
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
-
-  /**
-   * @return the dim
-   */
-  public CarbonDimension getDim() {
-    return dim;
-  }
-
-  /**
-   * @param dim the dim to set
-   */
-  public void setDim(CarbonDimension dim) {
-    this.dim = dim;
-  }
-
-  /**
-   * @return the aggList
-   */
-  public List<String> getAggList() {
-    return aggList;
-  }
-
-  /**
-   * @param aggList the aggList to set
-   */
-  public void setAggList(List<String> aggList) {
-    this.aggList = aggList;
-  }
-
-  /**
-   * @return the orderList
-   */
-  public List<Integer> getOrderList() {
-    return orderOfAggregation;
-  }
-
-  /**
-   * @param orderList the orderList to set
-   */
-  public void setOrderList(List<Integer> orderList) {
-    this.orderOfAggregation = orderList;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/QueryColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QueryColumn.java b/core/src/main/java/org/carbondata/query/carbon/model/QueryColumn.java
deleted file mode 100644
index 654996a..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/QueryColumn.java
+++ /dev/null
@@ -1,109 +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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-
-/**
- * query column  which will have information about column
- */
-public class QueryColumn implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -4222306600480181084L;
-
-  /**
-   * name of the column
-   */
-  protected String columnName;
-
-  /**
-   * sort order in which column output will be sorted default it will be none
-   */
-  private SortOrderType sortOrder = SortOrderType.NONE;
-
-  /**
-   * query order in which result of the query will be send
-   */
-  private int queryOrder;
-
-  /**
-   * aggregation function applied on column
-   */
-
-  private String aggregationFunction=CarbonCommonConstants.DUMMY;
-
-  public QueryColumn(String columnName) {
-    this.columnName = columnName;
-  }
-
-  /**
-   * @return the sortOrder
-   */
-  public SortOrderType getSortOrder() {
-    return sortOrder;
-  }
-
-  /**
-   * @param sortOrder the sortOrder to set
-   */
-  public void setSortOrder(SortOrderType sortOrder) {
-    this.sortOrder = sortOrder;
-  }
-
-  /**
-   * @return the columnName
-   */
-  public String getColumnName() {
-    return columnName;
-  }
-
-  /**
-   * @return the queryOrder
-   */
-  public int getQueryOrder() {
-    return queryOrder;
-  }
-
-  /**
-   * @param queryOrder the queryOrder to set
-   */
-  public void setQueryOrder(int queryOrder) {
-    this.queryOrder = queryOrder;
-  }
-
-  /**
-   * @return the aggregationFunction
-   */
-  public String getAggregateFunction() {
-    return aggregationFunction;
-  }
-
-  /**
-   * @param aggregationFunction the aggregationFunction to set
-   */
-  public void setAggregateFunction(String aggregationFunction) {
-    this.aggregationFunction = aggregationFunction;
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/QueryDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QueryDimension.java b/core/src/main/java/org/carbondata/query/carbon/model/QueryDimension.java
deleted file mode 100644
index 2948166..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/QueryDimension.java
+++ /dev/null
@@ -1,57 +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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-
-/**
- * query plan dimension which will holds the information about the query plan dimension
- * this is done to avoid heavy object serialization
- */
-public class QueryDimension extends QueryColumn implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = -8492704093776645651L;
-  /**
-   * actual dimension column
-   */
-  private transient CarbonDimension dimension;
-
-  public QueryDimension(String columName) {
-    super(columName);
-  }
-
-  /**
-   * @return the dimension
-   */
-  public CarbonDimension getDimension() {
-    return dimension;
-  }
-
-  /**
-   * @param dimension the dimension to set
-   */
-  public void setDimension(CarbonDimension dimension) {
-    this.dimension = dimension;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/QueryMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QueryMeasure.java b/core/src/main/java/org/carbondata/query/carbon/model/QueryMeasure.java
deleted file mode 100644
index f697ddd..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/QueryMeasure.java
+++ /dev/null
@@ -1,60 +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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-
-/**
- * query plan measure, this class will holds the information
- * about measure present in the query, this is done to avoid the serialization
- * of the heavy object
- */
-public class QueryMeasure extends QueryColumn implements Serializable {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 1035512411375495414L;
-
-  /**
-   * actual carbon measure object
-   */
-  private transient CarbonMeasure measure;
-
-  public QueryMeasure(String columName) {
-    super(columName);
-  }
-
-  /**
-   * @return the measure
-   */
-  public CarbonMeasure getMeasure() {
-    return measure;
-  }
-
-  /**
-   * @param measure the measure to set
-   */
-  public void setMeasure(CarbonMeasure measure) {
-    this.measure = measure;
-  }
-
-}


[42/56] [abbrv] incubator-carbondata git commit: Fixed data loading failure in Spark1.6 spark shell (#695)

Posted by jb...@apache.org.
Fixed data loading failure in Spark1.6 spark shell (#695)



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

Branch: refs/heads/master
Commit: 8777640274d87d2606ccb36ac971d07bf1029aef
Parents: 19590db
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Mon Jun 20 04:49:42 2016 +0530
Committer: Jacky Li <ja...@huawei.com>
Committed: Mon Jun 20 07:19:42 2016 +0800

----------------------------------------------------------------------
 assembly/pom.xml | 1 +
 1 file changed, 1 insertion(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/87776402/assembly/pom.xml
----------------------------------------------------------------------
diff --git a/assembly/pom.xml b/assembly/pom.xml
index 7c3bb8c..ef7d5d9 100644
--- a/assembly/pom.xml
+++ b/assembly/pom.xml
@@ -93,6 +93,7 @@
                 <exclude>META-INF/*.SF</exclude>
                 <exclude>META-INF/*.DSA</exclude>
                 <exclude>META-INF/*.RSA</exclude>
+                <exclude>META-INF/vfs-providers.xml</exclude>
               </excludes>
             </filter>
           </filters>


[44/56] [abbrv] incubator-carbondata git commit: [issue-698]fix error log message to be proper(#699)

Posted by jb...@apache.org.
[issue-698]fix error log message to be proper(#699)

fix table not found error message

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

Branch: refs/heads/master
Commit: 9559a44b4b4e6f84d9ef19b2cf9b19f4e0a9d5eb
Parents: 398250c
Author: Zhangshunyu <zh...@huawei.com>
Authored: Mon Jun 20 16:17:10 2016 +0800
Committer: david <Qi...@users.noreply.github.com>
Committed: Mon Jun 20 16:17:10 2016 +0800

----------------------------------------------------------------------
 .../spark/sql/execution/command/carbonTableSchema.scala   | 10 +++++-----
 1 file changed, 5 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9559a44b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index b70f9b1..655141c 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -1161,8 +1161,8 @@ private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) e
     val schemaName = getDB.getDatabaseName(alterTableModel.dbName, sqlContext)
     if (null == org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
       .getCarbonTable(schemaName + "_" + tableName)) {
-      logError("alter table failed. table not found: " + schemaName + "_" + tableName)
-      sys.error("alter table failed. table not found: " + schemaName + "_" + tableName)
+      logError("alter table failed. table not found: " + schemaName + "." + tableName)
+      sys.error("alter table failed. table not found: " + schemaName + "." + tableName)
     }
 
     val relation =
@@ -1418,9 +1418,9 @@ private[sql] case class LoadCube(
     }
     if (null == org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
       .getCarbonTable(schemaName + "_" + tableName)) {
-      logError("Data loading failed. table not found: " + schemaName + "_" + tableName)
-      LOGGER.audit("Data loading failed. table not found: " + schemaName + "_" + tableName)
-      sys.error("Data loading failed. table not found: " + schemaName + "_" + tableName)
+      logError("Data loading failed. table not found: " + schemaName + "." + tableName)
+      LOGGER.audit("Data loading failed. table not found: " + schemaName + "." + tableName)
+      sys.error("Data loading failed. table not found: " + schemaName + "." + tableName)
     }
     CarbonProperties.getInstance().addProperty("zookeeper.enable.lock", "false")
     val carbonLock = CarbonLockFactory.getCarbonLockObj(org.carbondata.core.


[24/56] [abbrv] incubator-carbondata git commit: Optimized detail query flow and cleanup (#691)

Posted by jb...@apache.org.
Optimized detail query flow and cleanup (#691)

* Optimizing detail query

* Optimizing detail query flow

* Optimizing detail query flow

* Optimized raw detail query to improve push up performance.

* Fixed bugs

* reverted wrong check in

* Rebased the code

* Removed aggregation from core

* Refactored core package and fixed test cases

* Fixed bugs

* Fixed review comments and deleted aggregate classes after merge from master

* Removed unused code

* Optimized scanner flow

* Optimized scanner flow

* Optimized scanning flow

* Optimized scanner flow

* Refactored code

* Refactored code

* Removed unused code

* Reverted unnecessary comment

* Reverted queryinterface package from core

* Removed queryinterface package from core

* Handled review comments

* Handled review comments

* Added assert


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

Branch: refs/heads/master
Commit: 29360501336ddc54a349257512fffb5bd8d87126
Parents: 656577d
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Sat Jun 18 21:39:54 2016 +0530
Committer: Jacky Li <ja...@huawei.com>
Committed: Sun Jun 19 00:09:54 2016 +0800

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |   6 +
 .../DirectDictionaryKeyGeneratorFactory.java    |   2 +-
 .../TimeStampDirectDictionaryGenerator.java     |   7 +
 .../aggregator/ScannedResultAggregator.java     |  43 ----
 .../impl/ListBasedResultAggregator.java         | 219 -----------------
 .../collector/ScannedResultCollector.java       |  43 ++++
 .../impl/ListBasedResultCollector.java          | 212 +++++++++++++++++
 .../executor/impl/DetailQueryExecutor.java      |   8 +-
 .../impl/DetailRawRecordQueryExecutor.java      |  10 +-
 .../executor/infos/BlockExecutionInfo.java      |  28 +--
 .../internal/InternalQueryExecutor.java         |   3 +-
 .../impl/InternalDetailQueryExecutor.java       | 130 -----------
 .../executor/internal/impl/QueryRunner.java     |  77 ------
 .../merger/AbstractScannedResultMerger.java     | 169 --------------
 .../carbon/merger/ScannedResultMerger.java      |  45 ----
 .../impl/UnSortedScannedResultMerger.java       |  34 ---
 .../query/carbon/model/QueryModel.java          |  14 ++
 .../processor/AbstractDataBlockIterator.java    | 126 ++++++++++
 .../processor/AbstractDataBlockProcessor.java   | 100 --------
 .../query/carbon/processor/BlockProcessor.java  |  36 ---
 .../impl/AggregateQueryBlockProcessor.java      |  62 -----
 .../processor/impl/DataBlockIteratorImpl.java   |  56 +++++
 .../impl/DetailQueryBlockProcessor.java         |  70 ------
 .../query/carbon/result/BatchRawResult.java     |  55 -----
 .../AbstractDetailQueryResultIterator.java      | 108 ++++-----
 .../iterator/DetailQueryResultIterator.java     |  71 +++---
 .../iterator/DetailRawQueryResultIterator.java  | 118 ----------
 .../impl/RawQueryResultPreparatorImpl.java      |  67 +++++-
 .../query/carbon/util/DataTypeUtil.java         |   4 +-
 .../merger/exception/ResultMergerException.java |  91 --------
 .../exception/DataProcessorException.java       |  91 --------
 .../queryinterface/filter/CarbonFilterInfo.java | 182 ---------------
 .../query/queryinterface/query/CarbonQuery.java | 233 -------------------
 .../query/impl/CarbonQueryImpl.java             | 223 ------------------
 .../query/metadata/AbstractCarbonLevel.java     |  57 -----
 .../queryinterface/query/metadata/Axis.java     |  61 -----
 .../query/metadata/CarbonCalculatedMeasure.java |  99 --------
 .../query/metadata/CarbonDimensionLevel.java    |  81 -------
 .../metadata/CarbonDimensionLevelFilter.java    | 217 -----------------
 .../query/metadata/CarbonLevel.java             |  83 -------
 .../query/metadata/CarbonLevelHolder.java       | 113 ---------
 .../query/metadata/CarbonMeasure.java           |  81 -------
 .../query/metadata/CarbonMeasureFilter.java     | 142 -----------
 .../query/metadata/CarbonMember.java            |  92 --------
 .../query/metadata/CarbonTuple.java             |  76 ------
 .../query/metadata/DSLTransformation.java       | 111 ---------
 .../queryinterface/query/metadata/TopCount.java | 110 ---------
 .../spark/merger/CarbonCompactionExecutor.java  |   1 +
 .../spark/partition/api/impl/PartitionImpl.java |  11 -
 .../api/impl/SampleDataPartitionerImpl.java     |  24 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  22 +-
 .../org/apache/spark/sql/CarbonOperators.scala  |  10 +-
 .../TimeStampDirectDictionaryGenerator_UT.java  |   6 +-
 53 files changed, 627 insertions(+), 3513 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
index 34b609c..15a21e3 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -733,6 +733,12 @@ public final class CarbonCommonConstants {
    */
   public static final String INMEMORY_REOCRD_SIZE = "carbon.inmemory.record.size";
   public static final int INMEMORY_REOCRD_SIZE_DEFAULT = 240000;
+
+  /**
+   * INMEMORY_REOCRD_SIZE
+   */
+  public static final String DETAIL_QUERY_BATCH_SIZE = "carbon.detail.batch.size";
+  public static final int DETAIL_QUERY_BATCH_SIZE_DEFAULT = 10000;
   /**
    * SPILL_OVER_DISK_PATH
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
index 3859663..f3633bf 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryKeyGeneratorFactory.java
@@ -43,7 +43,7 @@ public final class DirectDictionaryKeyGeneratorFactory {
     DirectDictionaryGenerator directDictionaryGenerator = null;
     switch (dataType) {
       case TIMESTAMP:
-        directDictionaryGenerator = new TimeStampDirectDictionaryGenerator();
+        directDictionaryGenerator = TimeStampDirectDictionaryGenerator.instance;
         break;
       default:
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
index 3954e14..d46725f 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
@@ -40,6 +40,13 @@ import static org.carbondata.core.keygenerator.directdictionary.timestamp.TimeSt
  */
 public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
 
+  private TimeStampDirectDictionaryGenerator() {
+
+  }
+
+  public static TimeStampDirectDictionaryGenerator instance =
+      new TimeStampDirectDictionaryGenerator();
+
   /**
    * Logger instance
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/aggregator/ScannedResultAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/ScannedResultAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/ScannedResultAggregator.java
deleted file mode 100644
index f5e4b5f..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/ScannedResultAggregator.java
+++ /dev/null
@@ -1,43 +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.carbondata.query.carbon.aggregator;
-
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Interface which will be used to aggregate the scan result
- */
-public interface ScannedResultAggregator {
-
-  /**
-   * Below method will be used to aggregate the scanned result
-   *
-   * @param scannedResult scanned result
-   * @return how many records was aggregated
-   */
-  int aggregateData(AbstractScannedResult scannedResult);
-
-  /**
-   * Below method will be used to get the aggregated result
-   *
-   * @return
-   */
-  Result getAggregatedResult();
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
deleted file mode 100644
index 10bf88c..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/aggregator/impl/ListBasedResultAggregator.java
+++ /dev/null
@@ -1,219 +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.carbondata.query.carbon.aggregator.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.carbon.aggregator.ScannedResultAggregator;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.impl.ListBasedResult;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * It is not a aggregator it is just a scanned result holder.
- *
- * @TODO change it to some other name
- */
-public class ListBasedResultAggregator implements ScannedResultAggregator {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ListBasedResultAggregator.class.getName());
-
-  /**
-   * to keep a track of number of row processed to handle limit push down in
-   * case of detail query scenario
-   */
-  private int rowCounter;
-
-  /**
-   * number of records asked in limit query if -1 then its either is
-   * detail+order by query or detail query
-   */
-  private int limit;
-
-  /**
-   * dimension values list
-   */
-  private List<ListBasedResultWrapper> listBasedResult;
-
-  /**
-   * restructuring info
-   */
-  private KeyStructureInfo restructureInfos;
-
-  /**
-   * table block execution infos
-   */
-  private BlockExecutionInfo tableBlockExecutionInfos;
-
-  private int[] measuresOrdinal;
-
-  /**
-   * to check whether measure exists in current table block or not this to
-   * handle restructuring scenario
-   */
-  private boolean[] isMeasureExistsInCurrentBlock;
-
-  /**
-   * default value of the measures in case of restructuring some measure wont
-   * be present in the table so in that default value will be used to
-   * aggregate the data for that measure columns
-   */
-  private Object[] measureDefaultValue;
-
-  /**
-   * measure datatypes.
-   */
-  private DataType[] measureDatatypes;
-
-  public ListBasedResultAggregator(BlockExecutionInfo blockExecutionInfos) {
-    limit = blockExecutionInfos.getLimit();
-    this.tableBlockExecutionInfos = blockExecutionInfos;
-    restructureInfos = blockExecutionInfos.getKeyStructureInfo();
-    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
-    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
-    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
-    this.measureDatatypes = tableBlockExecutionInfos.getAggregatorInfo().getMeasureDataTypes();
-  }
-
-  @Override
-  /**
-   * This method will add a record both key and value to list object
-   * it will keep track of how many record is processed, to handle limit scenario
-   * @param scanned result
-   *
-   */
-  public int aggregateData(AbstractScannedResult scannedResult) {
-    this.listBasedResult =
-        new ArrayList<>(limit == -1 ? scannedResult.numberOfOutputRows() : limit);
-    boolean isMsrsPresent = measureDatatypes.length > 0;
-    ByteArrayWrapper wrapper = null;
-    // scan the record and add to list
-    ListBasedResultWrapper resultWrapper;
-    while (scannedResult.hasNext() && (limit == -1 || rowCounter < limit)) {
-      resultWrapper = new ListBasedResultWrapper();
-      if(tableBlockExecutionInfos.isDimensionsExistInQuery()) {
-        wrapper = new ByteArrayWrapper();
-        wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
-        wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
-        wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
-        resultWrapper.setKey(wrapper);
-      } else {
-        scannedResult.incrementCounter();
-      }
-      if(isMsrsPresent) {
-        Object[] msrValues = new Object[measureDatatypes.length];
-        fillMeasureData(msrValues, scannedResult);
-        resultWrapper.setValue(msrValues);
-      }
-      listBasedResult.add(resultWrapper);
-      rowCounter++;
-    }
-    return rowCounter;
-  }
-
-  private void fillMeasureData(Object[] msrValues, AbstractScannedResult scannedResult) {
-    for (short i = 0; i < measuresOrdinal.length; i++) {
-      // if measure exists is block then pass measure column
-      // data chunk to the aggregator
-      if (isMeasureExistsInCurrentBlock[i]) {
-        msrValues[i] =
-            getMeasureData(scannedResult.getMeasureChunk(measuresOrdinal[i]),
-                scannedResult.getCurrenrRowId(),measureDatatypes[i]);
-      } else {
-        // if not then get the default value and use that value in aggregation
-        msrValues[i] = measureDefaultValue[i];
-      }
-    }
-  }
-
-  private Object getMeasureData(MeasureColumnDataChunk dataChunk, int index, DataType dataType) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      Object msrVal;
-      switch (dataType) {
-        case LONG:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-          break;
-        case DECIMAL:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-          break;
-        default:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      }
-      return DataTypeUtil.getMeasureDataBasedOnDataType(msrVal, dataType);
-    }
-    return null;
-  }
-
-  /**
-   * Below method will used to get the result
-   */
-  @Override public Result getAggregatedResult() {
-    Result<List<ListBasedResultWrapper>, Object> result = new ListBasedResult();
-    if (!tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
-      updateKeyWithLatestBlockKeyGenerator();
-    }
-    result.addScannedResult(listBasedResult);
-    return result;
-  }
-
-
-
-  /**
-   * Below method will be used to update the fixed length key with the
-   * latest block key generator
-   *
-   * @return updated block
-   */
-  private void updateKeyWithLatestBlockKeyGenerator() {
-    try {
-      long[] data = null;
-      ByteArrayWrapper key = null;
-      for (int i = 0; i < listBasedResult.size(); i++) {
-        // get the key
-        key = listBasedResult.get(i).getKey();
-        // unpack the key with table block key generator
-        data = tableBlockExecutionInfos.getBlockKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(), tableBlockExecutionInfos.getMaskedByteForBlock());
-        // packed the key with latest block key generator
-        // and generate the masked key for that key
-        key.setDictionaryKey(QueryUtil
-            .getMaskedKey(restructureInfos.getKeyGenerator().generateKey(data),
-                restructureInfos.getMaxKey(), restructureInfos.getMaskByteRanges(),
-                restructureInfos.getMaskByteRanges().length));
-        listBasedResult.get(i).setKey(key);
-      }
-    } catch (KeyGenException e) {
-      LOGGER.error(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java b/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java
new file mode 100644
index 0000000..9e5d401
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java
@@ -0,0 +1,43 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.query.carbon.collector;
+
+import org.carbondata.query.carbon.result.AbstractScannedResult;
+import org.carbondata.query.carbon.result.Result;
+
+/**
+ * Interface which will be used to aggregate the scan result
+ */
+public interface ScannedResultCollector {
+
+  /**
+   * Below method will be used to aggregate the scanned result
+   *
+   * @param scannedResult scanned result
+   * @return how many records was aggregated
+   */
+  int collectData(AbstractScannedResult scannedResult, int batchSize);
+
+  /**
+   * Below method will be used to get the aggregated result
+   *
+   * @return
+   */
+  Result getCollectedResult();
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java b/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java
new file mode 100644
index 0000000..30d33b8
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java
@@ -0,0 +1,212 @@
+/*
+ * 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.carbondata.query.carbon.collector.impl;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.query.carbon.collector.ScannedResultCollector;
+import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
+import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
+import org.carbondata.query.carbon.executor.util.QueryUtil;
+import org.carbondata.query.carbon.result.AbstractScannedResult;
+import org.carbondata.query.carbon.result.ListBasedResultWrapper;
+import org.carbondata.query.carbon.result.Result;
+import org.carbondata.query.carbon.result.impl.ListBasedResult;
+import org.carbondata.query.carbon.util.DataTypeUtil;
+import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
+
+/**
+ * It is not a collector it is just a scanned result holder.
+ *
+ */
+public class ListBasedResultCollector implements ScannedResultCollector {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ListBasedResultCollector.class.getName());
+
+  /**
+   * to keep a track of number of row processed to handle limit push down in
+   * case of detail query scenario
+   */
+  private int rowCounter;
+
+  /**
+   * dimension values list
+   */
+  private List<ListBasedResultWrapper> listBasedResult;
+
+  /**
+   * restructuring info
+   */
+  private KeyStructureInfo restructureInfos;
+
+  /**
+   * table block execution infos
+   */
+  private BlockExecutionInfo tableBlockExecutionInfos;
+
+  private int[] measuresOrdinal;
+
+  /**
+   * to check whether measure exists in current table block or not this to
+   * handle restructuring scenario
+   */
+  private boolean[] isMeasureExistsInCurrentBlock;
+
+  /**
+   * default value of the measures in case of restructuring some measure wont
+   * be present in the table so in that default value will be used to
+   * aggregate the data for that measure columns
+   */
+  private Object[] measureDefaultValue;
+
+  /**
+   * measure datatypes.
+   */
+  private DataType[] measureDatatypes;
+
+  public ListBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
+    this.tableBlockExecutionInfos = blockExecutionInfos;
+    restructureInfos = blockExecutionInfos.getKeyStructureInfo();
+    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
+    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
+    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
+    this.measureDatatypes = tableBlockExecutionInfos.getAggregatorInfo().getMeasureDataTypes();
+  }
+
+  @Override
+  /**
+   * This method will add a record both key and value to list object
+   * it will keep track of how many record is processed, to handle limit scenario
+   * @param scanned result
+   *
+   */
+  public int collectData(AbstractScannedResult scannedResult, int batchSize) {
+    this.listBasedResult =
+        new ArrayList<>(batchSize);
+    boolean isMsrsPresent = measureDatatypes.length > 0;
+    ByteArrayWrapper wrapper = null;
+    // scan the record and add to list
+    ListBasedResultWrapper resultWrapper;
+    int rowCounter = 0;
+    while (scannedResult.hasNext() && rowCounter < batchSize) {
+      resultWrapper = new ListBasedResultWrapper();
+      if(tableBlockExecutionInfos.isDimensionsExistInQuery()) {
+        wrapper = new ByteArrayWrapper();
+        wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
+        wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
+        wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
+        resultWrapper.setKey(wrapper);
+      } else {
+        scannedResult.incrementCounter();
+      }
+      if(isMsrsPresent) {
+        Object[] msrValues = new Object[measureDatatypes.length];
+        fillMeasureData(msrValues, scannedResult);
+        resultWrapper.setValue(msrValues);
+      }
+      listBasedResult.add(resultWrapper);
+      rowCounter++;
+    }
+    return rowCounter;
+  }
+
+  private void fillMeasureData(Object[] msrValues, AbstractScannedResult scannedResult) {
+    for (short i = 0; i < measuresOrdinal.length; i++) {
+      // if measure exists is block then pass measure column
+      // data chunk to the collector
+      if (isMeasureExistsInCurrentBlock[i]) {
+        msrValues[i] =
+            getMeasureData(scannedResult.getMeasureChunk(measuresOrdinal[i]),
+                scannedResult.getCurrenrRowId(),measureDatatypes[i]);
+      } else {
+        // if not then get the default value and use that value in aggregation
+        msrValues[i] = measureDefaultValue[i];
+      }
+    }
+  }
+
+  private Object getMeasureData(MeasureColumnDataChunk dataChunk, int index, DataType dataType) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      Object msrVal;
+      switch (dataType) {
+        case LONG:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+          break;
+        case DECIMAL:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+          break;
+        default:
+          msrVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+      }
+      return DataTypeUtil.getMeasureDataBasedOnDataType(msrVal, dataType);
+    }
+    return null;
+  }
+
+  /**
+   * Below method will used to get the result
+   */
+  @Override public Result getCollectedResult() {
+    Result<List<ListBasedResultWrapper>, Object> result = new ListBasedResult();
+    if (!tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
+      updateKeyWithLatestBlockKeyGenerator();
+    }
+    result.addScannedResult(listBasedResult);
+    return result;
+  }
+
+
+
+  /**
+   * Below method will be used to update the fixed length key with the
+   * latest block key generator
+   *
+   * @return updated block
+   */
+  private void updateKeyWithLatestBlockKeyGenerator() {
+    try {
+      long[] data = null;
+      ByteArrayWrapper key = null;
+      for (int i = 0; i < listBasedResult.size(); i++) {
+        // get the key
+        key = listBasedResult.get(i).getKey();
+        // unpack the key with table block key generator
+        data = tableBlockExecutionInfos.getBlockKeyGenerator()
+            .getKeyArray(key.getDictionaryKey(), tableBlockExecutionInfos.getMaskedByteForBlock());
+        // packed the key with latest block key generator
+        // and generate the masked key for that key
+        key.setDictionaryKey(QueryUtil
+            .getMaskedKey(restructureInfos.getKeyGenerator().generateKey(data),
+                restructureInfos.getMaxKey(), restructureInfos.getMaskByteRanges(),
+                restructureInfos.getMaskByteRanges().length));
+        listBasedResult.get(i).setKey(key);
+      }
+    } catch (KeyGenException e) {
+      LOGGER.error(e);
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
index b2f323c..0255cbb 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
@@ -23,11 +23,10 @@ import java.util.List;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.executor.internal.impl.InternalDetailQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
 import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
 import org.carbondata.query.carbon.result.iterator.DetailQueryResultIterator;
+import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPreparatorImpl;
 
 /**
  * Below class will be used to execute the detail query
@@ -39,10 +38,9 @@ public class DetailQueryExecutor extends AbstractQueryExecutor {
   @Override public CarbonIterator<Object[]> execute(QueryModel queryModel)
       throws QueryExecutionException {
     List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    InternalQueryExecutor queryExecutor = new InternalDetailQueryExecutor();
     return new ChunkRowIterator(
-        new DetailQueryResultIterator(blockExecutionInfoList, queryProperties, queryModel,
-            queryExecutor));
+        new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
+            new DetailQueryResultPreparatorImpl(queryProperties, queryModel)));
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
index d1967cd..e72c638 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
@@ -5,11 +5,10 @@ import java.util.List;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.executor.internal.impl.InternalDetailQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
 import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.iterator.DetailRawQueryResultIterator;
+import org.carbondata.query.carbon.result.iterator.DetailQueryResultIterator;
+import org.carbondata.query.carbon.result.preparator.impl.RawQueryResultPreparatorImpl;
 
 /**
  * Executor for raw records, it does not parse to actual data
@@ -19,8 +18,7 @@ public class DetailRawRecordQueryExecutor extends AbstractQueryExecutor<BatchRes
   @Override public CarbonIterator<BatchResult> execute(QueryModel queryModel)
       throws QueryExecutionException {
     List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    InternalQueryExecutor queryExecutor = new InternalDetailQueryExecutor();
-    return new DetailRawQueryResultIterator(blockExecutionInfoList, queryProperties, queryModel,
-        queryExecutor);
+    return new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
+        new RawQueryResultPreparatorImpl(queryProperties, queryModel));
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
index 7bed33d..202a932 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
@@ -26,7 +26,6 @@ import org.carbondata.core.carbon.datastore.IndexKey;
 import org.carbondata.core.carbon.datastore.block.AbstractIndex;
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.query.carbon.merger.ScannedResultMerger;
 import org.carbondata.query.filter.executer.FilterExecuter;
 
 /**
@@ -156,11 +155,6 @@ public class BlockExecutionInfo {
   private int[] noDictionaryBlockIndexes;
 
   /**
-   * to process the scanned result
-   */
-  private ScannedResultMerger scannedResultProcessor;
-
-  /**
    * key generator used for generating the table block fixed length key
    */
   private KeyGenerator blockKeyGenerator;
@@ -214,7 +208,7 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @param tableBlock the tableBlock to set
+   * @param blockIndex the tableBlock to set
    */
   public void setDataBlock(AbstractIndex blockIndex) {
     this.blockIndex = blockIndex;
@@ -410,7 +404,7 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @param restructureInfos the restructureInfos to set
+   * @param keyStructureInfo the restructureInfos to set
    */
   public void setKeyStructureInfo(KeyStructureInfo keyStructureInfo) {
     this.keyStructureInfo = keyStructureInfo;
@@ -424,7 +418,7 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @param sortInfos the sortInfos to set
+   * @param sortInfo the sortInfos to set
    */
   public void setSortInfo(SortInfo sortInfo) {
     this.sortInfo = sortInfo;
@@ -473,20 +467,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the scannedResultProcessor
-   */
-  public ScannedResultMerger getScannedResultProcessor() {
-    return scannedResultProcessor;
-  }
-
-  /**
-   * @param scannedResultProcessor the scannedResultProcessor to set
-   */
-  public void setScannedResultProcessor(ScannedResultMerger scannedResultProcessor) {
-    this.scannedResultProcessor = scannedResultProcessor;
-  }
-
-  /**
    * @return the filterEvaluatorTree
    */
   public FilterExecuter getFilterExecuterTree() {
@@ -494,7 +474,7 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @param filterEvaluatorTree the filterEvaluatorTree to set
+   * @param filterExecuterTree the filterEvaluatorTree to set
    */
   public void setFilterExecuterTree(FilterExecuter filterExecuterTree) {
     this.filterExecuterTree = filterExecuterTree;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java
index 7b3691c..089ee82 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/InternalQueryExecutor.java
@@ -20,6 +20,7 @@ package org.carbondata.query.carbon.executor.internal;
 
 import java.util.List;
 
+import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
@@ -42,5 +43,5 @@ public interface InternalQueryExecutor {
    * @throws QueryExecutionException
    */
   CarbonIterator<Result> executeQuery(List<BlockExecutionInfo> blockExecutionInfos,
-      int[] blockIndexToBeExecuted) throws QueryExecutionException;
+      int[] blockIndexToBeExecuted, FileHolder fileReader) throws QueryExecutionException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
deleted file mode 100644
index 866596f..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
+++ /dev/null
@@ -1,130 +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.carbondata.query.carbon.executor.internal.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.TimeUnit;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.merger.ScannedResultMerger;
-import org.carbondata.query.carbon.merger.impl.UnSortedScannedResultMerger;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Below Class will be used to execute the detail query
- */
-public class InternalDetailQueryExecutor implements InternalQueryExecutor {
-
-  /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(InternalDetailQueryExecutor.class.getName());
-
-  /**
-   * number of cores can be used to execute the query
-   */
-  private int numberOfCores;
-
-  public InternalDetailQueryExecutor() {
-
-    // below code will be used to update the number of cores based on number
-    // records we
-    // can keep in memory while executing the query execution
-    int recordSize = 0;
-    String defaultInMemoryRecordsSize =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.INMEMORY_REOCRD_SIZE);
-    if (null != defaultInMemoryRecordsSize) {
-      try {
-        recordSize = Integer.parseInt(defaultInMemoryRecordsSize);
-      } catch (NumberFormatException ne) {
-        LOGGER.error("Invalid inmemory records size. Using default value");
-        recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
-      }
-    }
-    numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    if (numberOfCores == 0) {
-      numberOfCores++;
-    }
-  }
-
-  /**
-   * Below method will be used to used to execute the detail query
-   * and it will return iterator over result
-   *
-   * @param executionInfos block execution info which will have all the properties
-   *                       required for query execution
-   * @param sliceIndexes   slice indexes to be executed
-   * @return query result
-   */
-  @Override public CarbonIterator<Result> executeQuery(
-      List<BlockExecutionInfo> executionInfos,
-      int[] sliceIndexes) throws QueryExecutionException {
-    long startTime = System.currentTimeMillis();
-    QueryRunner task;
-    ScannedResultMerger scannedResultProcessor =
-        new UnSortedScannedResultMerger(executionInfos.get(executionInfos.size() - 1),
-            sliceIndexes.length);
-    ExecutorService execService = Executors.newFixedThreadPool(numberOfCores);
-    List<Future> listFutureObjects = new ArrayList<Future>();
-    try {
-      for (int currentSliceIndex : sliceIndexes) {
-        if (currentSliceIndex == -1) {
-          continue;
-        }
-        executionInfos.get(currentSliceIndex).setScannedResultProcessor(scannedResultProcessor);
-        task = new QueryRunner(executionInfos.get(currentSliceIndex));
-        listFutureObjects.add(execService.submit(task));
-      }
-      execService.shutdown();
-      execService.awaitTermination(2, TimeUnit.DAYS);
-      LOGGER.info("Total time taken for scan " + (System.currentTimeMillis() - startTime));
-      for (Future future : listFutureObjects) {
-        try {
-          future.get();
-        } catch (ExecutionException e) {
-          throw new QueryExecutionException(e.getMessage());
-        }
-      }
-      return scannedResultProcessor.getQueryResultIterator();
-    } catch (QueryExecutionException exception) {
-      throw new QueryExecutionException(exception);
-    } catch (InterruptedException e) {
-      LOGGER.error(e, e.getMessage());
-      throw new QueryExecutionException(e);
-    } finally {
-      execService = null;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
deleted file mode 100644
index f341fa9..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
+++ /dev/null
@@ -1,77 +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.carbondata.query.carbon.executor.internal.impl;
-
-import java.util.concurrent.Callable;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.common.logging.impl.StandardLogService;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.BlockProcessor;
-import org.carbondata.query.carbon.processor.impl.DetailQueryBlockProcessor;
-
-/**
- * Class which will execute the query
- */
-public class QueryRunner implements Callable<Void> {
-
-  /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(QueryRunner.class.getName());
-  /**
-   * block processor
-   */
-  private BlockProcessor dataBlockProcessor;
-  /**
-   * file reader which will be used to execute the query
-   */
-  private FileHolder fileReader;
-  /**
-   * block execution info which is required to run the query
-   */
-  private BlockExecutionInfo blockExecutionInfo;
-
-  public QueryRunner(BlockExecutionInfo executionInfo) {
-    this.blockExecutionInfo = executionInfo;
-    this.fileReader = FileFactory.getFileHolder(executionInfo.getFileType());
-    // if detail query detail query processor will be used to process the
-    // block
-    dataBlockProcessor = new DetailQueryBlockProcessor(executionInfo, fileReader);
-  }
-
-  @Override public Void call() throws Exception {
-    StandardLogService
-        .setThreadName(blockExecutionInfo.getPartitionId(), blockExecutionInfo.getQueryId());
-    try {
-      this.dataBlockProcessor.processBlock();
-    } catch (Exception e) {
-      LOGGER.error(e);
-      throw new Exception(e.getMessage());
-    } finally {
-      this.fileReader.finish();
-    }
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java b/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
deleted file mode 100644
index 9358f42..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
+++ /dev/null
@@ -1,169 +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.carbondata.query.carbon.merger;
-
-import java.util.ArrayList;
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.impl.ListBasedResult;
-import org.carbondata.query.carbon.result.iterator.MemoryBasedResultIterator;
-
-/**
- * Class which processed the scanned result
- * Processing can be merging sorting
- */
-public abstract class AbstractScannedResultMerger implements ScannedResultMerger {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractScannedResultMerger.class.getName());
-  /**
-   * merging will done using thread
-   */
-  protected ExecutorService execService;
-
-  /**
-   * Merged scanned result which will merge all the result from all the blocks
-   * executor
-   */
-  protected Result mergedScannedResult;
-
-  /**
-   * scanned result list
-   */
-  protected List<Result> scannedResultList;
-
-  /**
-   * tableBlockExecutionInfo
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  /**
-   * max number of scanned result can keep in memory
-   */
-  private int maxNumberOfScannedResultList;
-
-  /**
-   * lockObject
-   */
-  private Object lockObject;
-
-  public AbstractScannedResultMerger(BlockExecutionInfo blockExecutionInfo,
-      int maxNumberOfScannedresultList) {
-
-    this.lockObject = new Object();
-    this.maxNumberOfScannedResultList = maxNumberOfScannedresultList;
-    execService = Executors.newFixedThreadPool(1);
-    scannedResultList = new ArrayList<Result>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    this.blockExecutionInfo = blockExecutionInfo;
-    initialiseResult();
-  }
-
-  /**
-   * for initializing the map based or list based result.
-   */
-  protected void initialiseResult() {
-    mergedScannedResult = new ListBasedResult();
-  }
-
-  /**
-   * Below method will be used to add the scanned result
-   * If number of scanned result in the list of more than
-   * the maxNumberOfScannedResultList than results present in the
-   * list will be merged to merged scanned result
-   *
-   * @param scannedResult
-   */
-  @Override public void addScannedResult(Result scannedResult) throws QueryExecutionException {
-    synchronized (this.lockObject) {
-      scannedResultList.add(scannedResult);
-      if ((scannedResultList.size() > maxNumberOfScannedResultList)) {
-        List<Result> localResult = scannedResultList;
-        scannedResultList = new ArrayList<Result>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        execService.submit(new MergerThread(localResult));
-      }
-    }
-  }
-
-  /**
-   * Below method will be used to merge the scanned result
-   *
-   * @param scannedResultList scanned result list
-   */
-  protected void mergeScannedResults(List<Result> scannedResultList) {
-    long start = System.currentTimeMillis();
-    LOGGER.debug("Started a slice result merging");
-
-    for (int i = 0; i < scannedResultList.size(); i++) {
-      mergedScannedResult.merge(scannedResultList.get(i));
-    }
-    LOGGER.debug("Finished current slice result merging in time (MS) " + (System.currentTimeMillis()
-            - start));
-  }
-
-  /**
-   * Below method will be used to get the final query
-   * return
-   *
-   * @return iterator over result
-   */
-  @Override public CarbonIterator<Result> getQueryResultIterator() throws QueryExecutionException {
-    execService.shutdown();
-    try {
-      execService.awaitTermination(1, TimeUnit.DAYS);
-    } catch (InterruptedException e1) {
-      LOGGER.error("Problem in thread termination" + e1.getMessage());
-    }
-    if (scannedResultList.size() > 0) {
-      mergeScannedResults(scannedResultList);
-      scannedResultList = null;
-    }
-    LOGGER.debug("Finished result merging from all slices");
-    return new MemoryBasedResultIterator(mergedScannedResult);
-  }
-
-  /**
-   * Thread class to merge the scanned result
-   */
-  private final class MergerThread implements Callable<Void> {
-    private List<Result> scannedResult;
-
-    private MergerThread(List<Result> scannedResult) {
-      this.scannedResult = scannedResult;
-    }
-
-    @Override public Void call() throws Exception {
-      mergeScannedResults(scannedResult);
-      return null;
-    }
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/merger/ScannedResultMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/merger/ScannedResultMerger.java b/core/src/main/java/org/carbondata/query/carbon/merger/ScannedResultMerger.java
deleted file mode 100644
index 19ed6cd..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/merger/ScannedResultMerger.java
+++ /dev/null
@@ -1,45 +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.carbondata.query.carbon.merger;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Interface for merging the scanned result
- */
-public interface ScannedResultMerger {
-
-  /**
-   * Below method will be used to add the scanned result
-   *
-   * @param scannedResult scanned result
-   * @throws QueryExecutionException throw exception in case of failure
-   */
-  void addScannedResult(Result scannedResult) throws QueryExecutionException;
-
-  /**
-   * Below method will be used to get the query result
-   *
-   * @return query result
-   * @throws QueryExecutionException throw exception in case of any failure
-   */
-  CarbonIterator<Result> getQueryResultIterator() throws QueryExecutionException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/merger/impl/UnSortedScannedResultMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/merger/impl/UnSortedScannedResultMerger.java b/core/src/main/java/org/carbondata/query/carbon/merger/impl/UnSortedScannedResultMerger.java
deleted file mode 100644
index 8a6c302..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/merger/impl/UnSortedScannedResultMerger.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.carbondata.query.carbon.merger.impl;
-
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.merger.AbstractScannedResultMerger;
-
-/**
- * Below class will be used merge the unsorted result
- */
-public class UnSortedScannedResultMerger extends AbstractScannedResultMerger {
-
-  public UnSortedScannedResultMerger(BlockExecutionInfo blockExecutionInfo,
-      int maxNumberOfScannedresultList) {
-    super(blockExecutionInfo, maxNumberOfScannedresultList);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
index 64b519a..92334db 100644
--- a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
+++ b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
@@ -140,6 +140,12 @@ public class QueryModel implements Serializable {
    */
   private CarbonTable table;
 
+  /**
+   * This is used only whne [forcedDetailRawQuery = true]. By default forcedDetailRawQuery returns
+   * dictionary values. But user wants in detail raw bytes the user set this field to true.
+   */
+  private boolean rawBytesDetailQuery;
+
   public QueryModel() {
     tableBlockInfos = new ArrayList<TableBlockInfo>();
     queryDimension = new ArrayList<QueryDimension>();
@@ -499,4 +505,12 @@ public class QueryModel implements Serializable {
   public void setColumnToDictionaryMapping(Map<String, Dictionary> columnToDictionaryMapping) {
     this.columnToDictionaryMapping = columnToDictionaryMapping;
   }
+
+  public boolean isRawBytesDetailQuery() {
+    return rawBytesDetailQuery;
+  }
+
+  public void setRawBytesDetailQuery(boolean rawBytesDetailQuery) {
+    this.rawBytesDetailQuery = rawBytesDetailQuery;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java
new file mode 100644
index 0000000..52b1bdf
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java
@@ -0,0 +1,126 @@
+/*
+ * 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.carbondata.query.carbon.processor;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.query.carbon.collector.ScannedResultCollector;
+import org.carbondata.query.carbon.collector.impl.ListBasedResultCollector;
+import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
+import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
+import org.carbondata.query.carbon.result.AbstractScannedResult;
+import org.carbondata.query.carbon.result.Result;
+import org.carbondata.query.carbon.scanner.BlockletScanner;
+import org.carbondata.query.carbon.scanner.impl.FilterScanner;
+import org.carbondata.query.carbon.scanner.impl.NonFilterScanner;
+
+/**
+ * This abstract class provides a skeletal implementation of the
+ * Block iterator.
+ */
+public abstract class AbstractDataBlockIterator extends CarbonIterator<Result> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName());
+  /**
+   * iterator which will be used to iterate over data blocks
+   */
+  protected CarbonIterator<DataRefNode> dataBlockIterator;
+
+  /**
+   * execution details
+   */
+  protected BlockExecutionInfo blockExecutionInfo;
+
+  /**
+   * result collector which will be used to aggregate the scanned result
+   */
+  protected ScannedResultCollector scannerResultAggregator;
+
+  /**
+   * processor which will be used to process the block processing can be
+   * filter processing or non filter processing
+   */
+  protected BlockletScanner blockletScanner;
+
+  /**
+   * to hold the data block
+   */
+  protected BlocksChunkHolder blocksChunkHolder;
+
+  /**
+   * batch size of result
+   */
+  protected int batchSize;
+
+  protected AbstractScannedResult scannedResult;
+
+  public AbstractDataBlockIterator(BlockExecutionInfo blockExecutionInfo,
+      FileHolder fileReader, int batchSize) {
+    this.blockExecutionInfo = blockExecutionInfo;
+    dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
+        blockExecutionInfo.getNumberOfBlockToScan());
+    blocksChunkHolder = new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(),
+        blockExecutionInfo.getTotalNumberOfMeasureBlock());
+    blocksChunkHolder.setFileReader(fileReader);
+
+    if (blockExecutionInfo.getFilterExecuterTree() != null) {
+      blockletScanner = new FilterScanner(blockExecutionInfo);
+    } else {
+      blockletScanner = new NonFilterScanner(blockExecutionInfo);
+    }
+
+    this.scannerResultAggregator =
+        new ListBasedResultCollector(blockExecutionInfo);
+    this.batchSize = batchSize;
+  }
+
+  public boolean hasNext() {
+    try {
+      if (scannedResult != null && scannedResult.hasNext()) {
+        return true;
+      } else {
+        scannedResult = getNextScannedResult();
+        while (scannedResult != null) {
+          if (scannedResult.hasNext()) {
+            return true;
+          }
+          scannedResult = getNextScannedResult();
+        }
+        return false;
+      }
+    } catch (QueryExecutionException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  private AbstractScannedResult getNextScannedResult() throws QueryExecutionException {
+    if (dataBlockIterator.hasNext()) {
+      blocksChunkHolder.setDataBlock(dataBlockIterator.next());
+      blocksChunkHolder.reset();
+      return blockletScanner.scanBlocklet(blocksChunkHolder);
+    }
+    return null;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
deleted file mode 100644
index fdcf6f1..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
+++ /dev/null
@@ -1,100 +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.carbondata.query.carbon.processor;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.aggregator.ScannedResultAggregator;
-import org.carbondata.query.carbon.aggregator.impl.ListBasedResultAggregator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.scanner.BlockletScanner;
-import org.carbondata.query.carbon.scanner.impl.FilterScanner;
-import org.carbondata.query.carbon.scanner.impl.NonFilterScanner;
-
-/**
- * This class provides a skeletal implementation of the
- * {@link BlockProcessor} interface to minimize the effort required to
- * implement this interface.
- */
-public abstract class AbstractDataBlockProcessor implements BlockProcessor {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractDataBlockProcessor.class.getName());
-  /**
-   * iterator which will be used to iterate over data blocks
-   */
-  protected CarbonIterator<DataRefNode> dataBlockIterator;
-
-  /**
-   * execution details
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  /**
-   * result aggregator which will be used to aggregate the scanned result
-   */
-  protected ScannedResultAggregator scannerResultAggregator;
-
-  /**
-   * processor which will be used to process the block processing can be
-   * filter processing or non filter processing
-   */
-  protected BlockletScanner blockletScanner;
-
-  /**
-   * to hold the data block
-   */
-  protected BlocksChunkHolder blocksChunkHolder;
-
-  public AbstractDataBlockProcessor(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader) {
-    this.blockExecutionInfo = blockExecutionInfo;
-    dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
-        blockExecutionInfo.getNumberOfBlockToScan());
-    blocksChunkHolder = new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(),
-        blockExecutionInfo.getTotalNumberOfMeasureBlock());
-    blocksChunkHolder.setFileReader(fileReader);
-
-    if (blockExecutionInfo.getFilterExecuterTree() != null) {
-      blockletScanner = new FilterScanner(blockExecutionInfo);
-    } else {
-      blockletScanner = new NonFilterScanner(blockExecutionInfo);
-    }
-
-    this.scannerResultAggregator =
-        new ListBasedResultAggregator(blockExecutionInfo);
-  }
-
-  /**
-   * Below method will be used to add the scanned result to scanned result
-   * processor
-   */
-  protected void finishScanning() {
-    try {
-      this.blockExecutionInfo.getScannedResultProcessor()
-          .addScannedResult(scannerResultAggregator.getAggregatedResult());
-    } catch (QueryExecutionException e) {
-      LOGGER.error(e,
-          "Problem while adding the result to Scanned Result Processor");
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/processor/BlockProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/BlockProcessor.java b/core/src/main/java/org/carbondata/query/carbon/processor/BlockProcessor.java
deleted file mode 100644
index 84e2d5f..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/BlockProcessor.java
+++ /dev/null
@@ -1,36 +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.carbondata.query.carbon.processor;
-
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-
-/**
- * Scanner interface which will be used
- * to scan the blocks.
- */
-public interface BlockProcessor {
-
-  /**
-   * Below method can be used to scan the block based on the query execution infos
-   *
-   * @throws QueryExecutionException
-   */
-  void processBlock() throws QueryExecutionException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/processor/impl/AggregateQueryBlockProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/impl/AggregateQueryBlockProcessor.java b/core/src/main/java/org/carbondata/query/carbon/processor/impl/AggregateQueryBlockProcessor.java
deleted file mode 100644
index 260b894..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/impl/AggregateQueryBlockProcessor.java
+++ /dev/null
@@ -1,62 +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.carbondata.query.carbon.processor.impl;
-
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.AbstractDataBlockProcessor;
-
-/**
- * Below class will be used to process the blocks for
- * aggregated query
- */
-public class AggregateQueryBlockProcessor extends AbstractDataBlockProcessor {
-
-  /**
-   * AggregateQueryScanner constructor
-   *
-   * @param blockExecutionInfos
-   */
-  public AggregateQueryBlockProcessor(BlockExecutionInfo tableBlockExecutionInfos,
-      FileHolder fileReader) {
-    super(tableBlockExecutionInfos, fileReader);
-  }
-
-  /**
-   * Below method will be used to scan the block
-   * then it will call processor to process the data
-   * and the it will call aggregator to aggregate the data
-   * it will call finish once all the blocks of a table is scanned
-   *
-   * @throws QueryExecutionException
-   */
-  @Override public void processBlock() throws QueryExecutionException {
-    while (dataBlockIterator.hasNext()) {
-      try {
-        blocksChunkHolder.setDataBlock(dataBlockIterator.next());
-        blocksChunkHolder.reset();
-        this.scannerResultAggregator.aggregateData(blockletScanner.scanBlocklet(blocksChunkHolder));
-      } catch (Exception e) {
-        throw new QueryExecutionException(e);
-      }
-    }
-    finishScanning();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.java b/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.java
new file mode 100644
index 0000000..ade1965
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.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.carbondata.query.carbon.processor.impl;
+
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
+import org.carbondata.query.carbon.processor.AbstractDataBlockIterator;
+import org.carbondata.query.carbon.result.Result;
+
+/**
+ * Below class will be used to process the block for detail query
+ */
+public class DataBlockIteratorImpl extends AbstractDataBlockIterator {
+
+  /**
+   * DataBlockIteratorImpl Constructor
+   *
+   * @param blockExecutionInfo execution information
+   */
+  public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo,
+      FileHolder fileReader, int batchSize) {
+    super(blockExecutionInfo, fileReader, batchSize);
+  }
+
+  /**
+   * It scans the block and returns the result with @batchSize
+   *
+   * @return Result of @batchSize
+   */
+  public Result next() {
+    this.scannerResultAggregator.collectData(scannedResult, batchSize);
+    Result result = this.scannerResultAggregator.getCollectedResult();
+    while (result.size() < batchSize && hasNext()) {
+      this.scannerResultAggregator.collectData(scannedResult, batchSize-result.size());
+      result.merge(this.scannerResultAggregator.getCollectedResult());
+    }
+    return result;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/processor/impl/DetailQueryBlockProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/impl/DetailQueryBlockProcessor.java b/core/src/main/java/org/carbondata/query/carbon/processor/impl/DetailQueryBlockProcessor.java
deleted file mode 100644
index b079a84..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/impl/DetailQueryBlockProcessor.java
+++ /dev/null
@@ -1,70 +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.carbondata.query.carbon.processor.impl;
-
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.AbstractDataBlockProcessor;
-
-/**
- * Below class will be used to process the block for detail query
- */
-public class DetailQueryBlockProcessor extends AbstractDataBlockProcessor {
-
-  /**
-   * counter for number of records processed
-   */
-  private int counter;
-
-  /**
-   * DetailQueryScanner Constructor
-   *
-   * @param blockExecutionInfo execution information
-   */
-  public DetailQueryBlockProcessor(BlockExecutionInfo blockExecutionInfo, FileHolder fileReader) {
-    super(blockExecutionInfo, fileReader);
-  }
-
-  /**
-   * Below method will be used scan the blocks and then process the scanned blocks
-   * as its a detail query so its will use dummy aggregator
-   * to aggregate the data.
-   * This scanner will handle the limit scenario if detail query is without order by.
-   * In case of detail query once one block is process it will pass to scanned result processor
-   * as in this case number of records will be more and it will take more memory
-   *
-   * @throws QueryExecutionException
-   */
-  @Override public void processBlock() throws QueryExecutionException {
-
-    while (dataBlockIterator.hasNext()) {
-      blocksChunkHolder.setDataBlock(dataBlockIterator.next());
-      blocksChunkHolder.reset();
-      counter += this.scannerResultAggregator
-          .aggregateData(blockletScanner.scanBlocklet(blocksChunkHolder));
-      //      finishScanning();
-      if (blockExecutionInfo.getLimit() != -1 && counter >= blockExecutionInfo.getLimit()) {
-        break;
-      }
-    }
-    finishScanning();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
index 4aa2adf..b00c021 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
@@ -19,21 +19,11 @@
 
 package org.carbondata.query.carbon.result;
 
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QuerySchemaInfo;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
 /**
  * Below class holds the query result of batches.
  */
 public class BatchRawResult extends BatchResult {
 
-  private QuerySchemaInfo querySchemaInfo;
-
   /**
    * This method will return one row at a time based on the counter given.
    * @param counter
@@ -42,51 +32,6 @@ public class BatchRawResult extends BatchResult {
   public Object[] getRawRow(int counter) {
     return rows[counter];
   }
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Object[] next() {
-    return parseData();
-  }
-
-  private Object[] parseData() {
-    int[] order = querySchemaInfo.getQueryReverseOrder();
-    Object[] row = rows[counter];
-    ByteArrayWrapper key = (ByteArrayWrapper) row[0];
-    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
-    Object[] parsedData = new Object[queryDimensions.length + row.length - 1];
-    if(key != null) {
-      long[] surrogateResult = querySchemaInfo.getKeyGenerator()
-          .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
-      int noDictionaryColumnIndex = 0;
-      for (int i = 0; i < queryDimensions.length; i++) {
-        if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
-          parsedData[order[i]] = DataTypeUtil.getDataBasedOnDataType(
-              new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
-              queryDimensions[i].getDimension().getDataType());
-        } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-          DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-              .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
-          parsedData[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
-              (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
-        } else {
-          parsedData[order[i]] =
-              (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
-        }
-      }
-    }
-    for (int i = 0; i < row.length - 1; i++) {
-      parsedData[order[i + queryDimensions.length]] = row[i + 1];
-    }
-    counter++;
-    return parsedData;
-  }
-
-  public void setQuerySchemaInfo(QuerySchemaInfo querySchemaInfo) {
-    this.querySchemaInfo = querySchemaInfo;
-  }
 
   /**
    * For getting the total size.


[07/56] [abbrv] incubator-carbondata git commit: [Issue 618]Supported Spark 1.6 in Carbondata (#670)

Posted by jb...@apache.org.
[Issue 618]Supported Spark 1.6 in Carbondata (#670)

* [Issue-660] Show segments query should not fail, if table name is case insensitive (#662)

* Show segments should not fail, if table name is case insensitive

* Corrected test case

* [issue-656] fix load data when int column contains integer.min_value (#657)

* load data when int column contains min Integer

* fixed test case

*  fix test bigint

*  fix test bigint

* removed no used DATA_BIGINT case

* removed no used condition for unCompressMaxMin

* [issue- 664] select count(joinDate) from table_x is failing for direct dictionary column (#665)

* Supported Spark 1.6 by changing aggregation interfaces

* Fixed compile issue after rebase

* optmizing the flow with unsafe row

* Fixed bugs in push up

* Fixed compiler issues after rebasing

* Fixed merging issue after rebase

* Fixed scan query pushdown


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

Branch: refs/heads/master
Commit: ead0076b077873a776e0e02273f587500ee713e7
Parents: afb6ed4
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Fri Jun 17 05:52:03 2016 +0530
Committer: Jacky Li <ja...@huawei.com>
Committed: Fri Jun 17 08:22:03 2016 +0800

----------------------------------------------------------------------
 core/pom.xml                                    |  33 -
 .../core/carbon/metadata/CarbonMetadata.java    |  19 +-
 .../compression/ValueCompressonHolder.java      |   1 +
 .../TimeStampDirectDictionaryGenerator.java     |   4 +-
 .../core/util/ValueCompressionUtil.java         |  53 +-
 ...bstractDistinctCountAggregatorObjectSet.java | 103 ---
 .../aggregator/impl/AbstractMaxAggregator.java  |  60 --
 .../impl/AbstractMeasureAggregatorDummy.java    |  69 --
 .../impl/AbstractMeasureAggregatorMaxMin.java   |   5 +-
 .../aggregator/impl/AbstractMinAggregator.java  |  60 --
 .../impl/AvgBigDecimalAggregator.java           | 214 -----
 .../aggregator/impl/AvgDoubleAggregator.java    | 206 -----
 .../aggregator/impl/AvgLongAggregator.java      | 191 -----
 .../query/aggregator/impl/CountAggregator.java  | 204 -----
 .../impl/DistinctCountAggregator.java           | 319 --------
 .../impl/DistinctCountAggregatorObjectSet.java  |  96 ---
 ...tinctCountBigDecimalAggregatorObjectSet.java |  86 --
 .../DistinctCountLongAggregatorObjectSet.java   |  89 --
 .../impl/DistinctStringCountAggregator.java     | 165 ----
 .../impl/DummyBigDecimalAggregator.java         |  63 --
 .../aggregator/impl/DummyDoubleAggregator.java  |  64 --
 .../aggregator/impl/DummyLongAggregator.java    |  59 --
 .../query/aggregator/impl/MaxAggregator.java    |  68 --
 .../impl/MaxBigDecimalAggregator.java           |  67 --
 .../aggregator/impl/MaxLongAggregator.java      |  67 --
 .../query/aggregator/impl/MinAggregator.java    |  66 --
 .../impl/MinBigDecimalAggregator.java           |  66 --
 .../aggregator/impl/MinLongAggregator.java      |  66 --
 .../impl/SumBigDecimalAggregator.java           | 178 ----
 .../impl/SumDistinctBigDecimalAggregator.java   | 232 ------
 .../impl/SumDistinctDoubleAggregator.java       | 230 ------
 .../impl/SumDistinctLongAggregator.java         | 220 -----
 .../aggregator/impl/SumDoubleAggregator.java    | 178 ----
 .../aggregator/impl/SumLongAggregator.java      | 164 ----
 .../impl/avg/AbstractAvgAggregator.java         |  28 +
 .../impl/avg/AvgBigDecimalAggregator.java       | 218 +++++
 .../impl/avg/AvgDoubleAggregator.java           | 210 +++++
 .../aggregator/impl/avg/AvgLongAggregator.java  | 195 +++++
 .../aggregator/impl/count/CountAggregator.java  | 204 +++++
 ...bstractDistinctCountAggregatorObjectSet.java | 103 +++
 .../impl/distinct/DistinctCountAggregator.java  | 319 ++++++++
 .../DistinctCountAggregatorObjectSet.java       |  96 +++
 ...tinctCountBigDecimalAggregatorObjectSet.java |  86 ++
 .../DistinctCountLongAggregatorObjectSet.java   |  89 ++
 .../distinct/DistinctStringCountAggregator.java | 165 ++++
 .../SumDistinctBigDecimalAggregator.java        | 233 ++++++
 .../distinct/SumDistinctDoubleAggregator.java   | 231 ++++++
 .../distinct/SumDistinctLongAggregator.java     | 221 +++++
 .../dummy/AbstractMeasureAggregatorDummy.java   |  70 ++
 .../impl/dummy/DummyBigDecimalAggregator.java   |  63 ++
 .../impl/dummy/DummyDoubleAggregator.java       |  64 ++
 .../impl/dummy/DummyLongAggregator.java         |  59 ++
 .../impl/max/AbstractMaxAggregator.java         |  61 ++
 .../aggregator/impl/max/MaxAggregator.java      |  68 ++
 .../impl/max/MaxBigDecimalAggregator.java       |  67 ++
 .../aggregator/impl/max/MaxLongAggregator.java  |  67 ++
 .../impl/min/AbstractMinAggregator.java         |  61 ++
 .../aggregator/impl/min/MinAggregator.java      |  66 ++
 .../impl/min/MinBigDecimalAggregator.java       |  66 ++
 .../aggregator/impl/min/MinLongAggregator.java  |  66 ++
 .../impl/sum/SumBigDecimalAggregator.java       | 179 ++++
 .../impl/sum/SumDoubleAggregator.java           | 179 ++++
 .../aggregator/impl/sum/SumLongAggregator.java  | 165 ++++
 .../util/MeasureAggregatorFactory.java          |  44 +-
 .../DirectDictionaryDimensionAggregator.java    | 114 +++
 .../impl/InternalCountStartQueryExecutor.java   |   2 +-
 .../query/carbon/executor/util/QueryUtil.java   |   9 +-
 .../impl/QueryResultPreparatorImpl.java         |   6 +-
 .../examples/GenerateDictionaryExample.scala    |   2 +-
 .../spark/sql/common/util/QueryTest.scala       |   7 +-
 .../allqueries/AllDataTypesTestCase1.scala      |  31 +-
 .../allqueries/AllDataTypesTestCase2.scala      |  43 +-
 .../allqueries/AllDataTypesTestCase6.scala      |  14 +-
 integration/spark/pom.xml                       |  11 -
 .../org/apache/spark/sql/CarbonAggregate.scala  | 209 -----
 .../spark/sql/CarbonCatalystOperators.scala     | 289 +++----
 .../org/apache/spark/sql/CarbonContext.scala    |   3 +-
 .../spark/sql/CarbonDatasourceRelation.scala    |  11 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |  15 +-
 .../org/apache/spark/sql/CarbonOperators.scala  | 270 ++++---
 .../apache/spark/sql/CarbonRawAggregate.scala   | 239 ------
 .../apache/spark/sql/CarbonRawOperators.scala   |  55 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |   6 +-
 .../sql/SparkUnknownCarbonAggregator.scala      | 179 ----
 .../sql/agg/CarbonAggregationExpression.scala   |  50 ++
 .../apache/spark/sql/agg/CarbonAverage.scala    |  89 ++
 .../org/apache/spark/sql/agg/CarbonCount.scala  |  77 ++
 .../execution/command/carbonTableSchema.scala   | 254 +++---
 .../spark/sql/hive/CarbonMetastoreCatalog.scala | 134 ++-
 .../spark/sql/hive/CarbonRawStrategies.scala    |  78 +-
 .../spark/sql/hive/CarbonStrategies.scala       | 171 ++--
 .../apache/spark/sql/hive/CarbonStrategy.scala  |   2 +-
 .../spark/sql/optimizer/CarbonOptimizer.scala   |  27 +-
 .../org/carbondata/spark/CarbonFilters.scala    |  26 +-
 .../org/carbondata/spark/CarbonOption.scala     |   2 +-
 .../scala/org/carbondata/spark/KeyVal.scala     |   8 +
 .../carbondata/spark/agg/CarbonAggregates.scala | 807 -------------------
 .../spark/agg/MeasureAggregatorUDT.scala        |  55 --
 .../spark/rdd/CarbonDataRDDFactory.scala        |  56 +-
 .../spark/rdd/CarbonRawQueryRDD.scala           |  12 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |  20 +-
 .../spark/util/GlobalDictionaryUtil.scala       |   2 +-
 .../src/test/resources/datawithmaxinteger.csv   |  12 +
 .../test/resources/datawithmaxmininteger.csv    |  13 +
 .../src/test/resources/datawithmininteger.csv   |  12 +
 .../spark/sql/common/util/QueryTest.scala       |   8 +-
 .../TestLoadDataWithMaxMinInteger.scala         |  98 +++
 .../AllDataTypesTestCaseAggregate.scala         |   7 +-
 .../DataCompactionNoDictionaryTest.scala        |  56 +-
 .../dataretention/DataRetentionTestCase.scala   |  13 +-
 .../deleteTable/TestDeleteTableNewDDL.scala     |  23 +
 ...estampDataTypeDirectDictionaryTestCase.scala |   6 +
 .../AutoHighCardinalityIdentifyTestCase.scala   |   2 +-
 .../util/GlobalDictionaryUtilTestCase.scala     |   8 +-
 pom.xml                                         |   2 +-
 processing/pom.xml                              |  11 -
 .../store/CarbonDataFileAttributes.java         |   4 +-
 117 files changed, 5119 insertions(+), 6049 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/pom.xml
----------------------------------------------------------------------
diff --git a/core/pom.xml b/core/pom.xml
index 6140353..bfb13a6 100644
--- a/core/pom.xml
+++ b/core/pom.xml
@@ -60,39 +60,6 @@
       <version>${kettle.version}</version>
     </dependency>
     <dependency>
-      <groupId>eigenbase</groupId>
-      <artifactId>eigenbase-xom</artifactId>
-      <version>1.3.4</version>
-      <exclusions>
-        <exclusion>
-          <groupId>*</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>eigenbase</groupId>
-      <artifactId>eigenbase-properties</artifactId>
-      <version>1.1.4</version>
-      <exclusions>
-        <exclusion>
-          <groupId>*</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
-      <groupId>eigenbase</groupId>
-      <artifactId>eigenbase-resgen</artifactId>
-      <version>1.3.4</version>
-      <exclusions>
-        <exclusion>
-          <groupId>*</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>it.unimi.dsi</groupId>
       <artifactId>fastutil</artifactId>
       <version>6.5.0</version>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/core/carbon/metadata/CarbonMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/CarbonMetadata.java b/core/src/main/java/org/carbondata/core/carbon/metadata/CarbonMetadata.java
index a473048..488f548 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/CarbonMetadata.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/CarbonMetadata.java
@@ -56,7 +56,7 @@ public final class CarbonMetadata {
    * @param tableUniquName
    */
   public void removeTable(String tableUniquName) {
-    tableInfoMap.remove(tableUniquName);
+    tableInfoMap.remove(convertToLowerCase(tableUniquName));
   }
 
   /**
@@ -68,7 +68,7 @@ public final class CarbonMetadata {
    * @param carbonTable
    */
   public void addCarbonTable(CarbonTable carbonTable) {
-    tableInfoMap.put(carbonTable.getTableUniqueName(), carbonTable);
+    tableInfoMap.put(convertToLowerCase(carbonTable.getTableUniqueName()), carbonTable);
   }
 
   /**
@@ -77,12 +77,12 @@ public final class CarbonMetadata {
    * @param tableInfo
    */
   public void loadTableMetadata(TableInfo tableInfo) {
-    CarbonTable carbonTable = tableInfoMap.get(tableInfo.getTableUniqueName());
+    CarbonTable carbonTable = tableInfoMap.get(convertToLowerCase(tableInfo.getTableUniqueName()));
     if (null == carbonTable || carbonTable.getTableLastUpdatedTime() < tableInfo
         .getLastUpdatedTime()) {
       carbonTable = new CarbonTable();
       carbonTable.loadCarbonTable(tableInfo);
-      tableInfoMap.put(tableInfo.getTableUniqueName(), carbonTable);
+      tableInfoMap.put(convertToLowerCase(tableInfo.getTableUniqueName()), carbonTable);
     }
   }
 
@@ -93,7 +93,7 @@ public final class CarbonMetadata {
    * @return
    */
   public CarbonTable getCarbonTable(String tableUniqueName) {
-    return tableInfoMap.get(tableUniqueName);
+    return tableInfoMap.get(convertToLowerCase(tableUniqueName));
   }
 
   /**
@@ -104,6 +104,15 @@ public final class CarbonMetadata {
   }
 
   /**
+   * returns the given string in lowercase
+   * @param table
+   * @return
+   */
+  public String convertToLowerCase(String table) {
+    return table.toLowerCase();
+  }
+
+  /**
    * method will return dimension instance based on the column identifier
    * and table instance passed to it.
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java b/core/src/main/java/org/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
index 64591d5..89bf334 100644
--- a/core/src/main/java/org/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/store/compression/ValueCompressonHolder.java
@@ -88,6 +88,7 @@ public final class ValueCompressonHolder {
         break;
 
       case DATA_LONG:
+      case DATA_BIGINT:
 
         value.setValue(longCompressor.unCompress(data));
         break;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
index d37dcf1..3954e14 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
@@ -34,8 +34,6 @@ import static org.carbondata.core.keygenerator.directdictionary.timestamp.TimeSt
 import static org.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampGranularityConstants.TIME_GRAN_MIN;
 import static org.carbondata.core.keygenerator.directdictionary.timestamp.TimeStampGranularityConstants.TIME_GRAN_SEC;
 
-import org.apache.spark.sql.columnar.TIMESTAMP;
-
 /**
  * The class provides the method to generate dictionary key and getting the actual value from
  * the dictionaryKey for direct dictionary column for TIMESTAMP type.
@@ -119,7 +117,7 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
     try {
       dateToStr = timeParser.parse(memberStr);
     } catch (ParseException e) {
-      LOGGER.error("Cannot convert" + TIMESTAMP.toString() + " to Time/Long type value"
+      LOGGER.error("Cannot convert" + memberStr + " to Time/Long type value"
           + e.getMessage());
     }
     if (null == dateToStr) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/core/util/ValueCompressionUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/ValueCompressionUtil.java b/core/src/main/java/org/carbondata/core/util/ValueCompressionUtil.java
index fdea2c8..ac2281d 100644
--- a/core/src/main/java/org/carbondata/core/util/ValueCompressionUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/ValueCompressionUtil.java
@@ -30,9 +30,7 @@ import org.carbondata.core.datastorage.store.compression.ValueCompressonHolder;
 import org.carbondata.core.datastorage.store.compression.type.UnCompressByteArray;
 import org.carbondata.core.datastorage.store.compression.type.UnCompressDefaultLong;
 import org.carbondata.core.datastorage.store.compression.type.UnCompressMaxMinByte;
-import org.carbondata.core.datastorage.store.compression.type.UnCompressMaxMinByteForLong;
 import org.carbondata.core.datastorage.store.compression.type.UnCompressMaxMinDefault;
-import org.carbondata.core.datastorage.store.compression.type.UnCompressMaxMinDefaultLong;
 import org.carbondata.core.datastorage.store.compression.type.UnCompressMaxMinFloat;
 import org.carbondata.core.datastorage.store.compression.type.UnCompressMaxMinInt;
 import org.carbondata.core.datastorage.store.compression.type.UnCompressMaxMinLong;
@@ -150,7 +148,8 @@ public final class ValueCompressionUtil {
         return new CompressionFinder(COMPRESSION_TYPE.CUSTOM_BIGDECIMAL, DataType.DATA_BYTE,
             DataType.DATA_BYTE);
       case 'l':
-        return new CompressionFinder(COMPRESSION_TYPE.NONE, DataType.DATA_LONG, DataType.DATA_LONG);
+        return new CompressionFinder(COMPRESSION_TYPE.NONE,
+                DataType.DATA_BIGINT, DataType.DATA_BIGINT);
       default:
         break;
     }
@@ -160,6 +159,10 @@ public final class ValueCompressionUtil {
           getDataType((double) maxValue - (double) minValue, decimal, dataTypeSelected))) {
         return new CompressionFinder(COMPRESSION_TYPE.MAX_MIN, DataType.DATA_DOUBLE,
             getDataType((double) maxValue - (double) minValue, decimal, dataTypeSelected));
+      } else if (getSize(getDataType((double) maxValue, decimal, dataTypeSelected)) < getSize(
+              getDataType((double) maxValue - (double) minValue, decimal, dataTypeSelected))) {
+        return new CompressionFinder(COMPRESSION_TYPE.NONE, DataType.DATA_DOUBLE,
+                getDataType((double) maxValue - (double) minValue, decimal, dataTypeSelected));
       } else {
         return new CompressionFinder(COMPRESSION_TYPE.NONE, DataType.DATA_DOUBLE,
             getDataType((double) maxValue, decimal, dataTypeSelected));
@@ -311,6 +314,7 @@ public final class ValueCompressionUtil {
         return intResult;
 
       case DATA_LONG:
+      case DATA_BIGINT:
 
         long[] longResult = new long[value.length];
 
@@ -547,7 +551,7 @@ public final class ValueCompressionUtil {
    */
   public static ValueCompressonHolder.UnCompressValue unCompressNone(DataType compDataType,
       DataType actualDataType) {
-    if (actualDataType == DataType.DATA_LONG) {
+    if (actualDataType == DataType.DATA_BIGINT) {
       return new UnCompressDefaultLong();
     } else {
       switch (compDataType) {
@@ -574,7 +578,6 @@ public final class ValueCompressionUtil {
         default:
 
           return new UnCompressNoneDefault();
-
       }
     }
   }
@@ -584,42 +587,31 @@ public final class ValueCompressionUtil {
    */
   public static ValueCompressonHolder.UnCompressValue unCompressMaxMin(DataType compDataType,
       DataType actualDataType) {
-    if (actualDataType == DataType.DATA_LONG) {
-      switch (compDataType) {
-        case DATA_BYTE:
-          return new UnCompressMaxMinByteForLong();
-        case DATA_LONG:
-          return new UnCompressMaxMinDefaultLong();
-        default:
-          return new UnCompressMaxMinDefaultLong();
-      }
-    } else {
-      switch (compDataType) {
-        case DATA_BYTE:
+    switch (compDataType) {
+      case DATA_BYTE:
 
-          return new UnCompressMaxMinByte();
+        return new UnCompressMaxMinByte();
 
-        case DATA_SHORT:
+      case DATA_SHORT:
 
-          return new UnCompressMaxMinShort();
+        return new UnCompressMaxMinShort();
 
-        case DATA_INT:
+      case DATA_INT:
 
-          return new UnCompressMaxMinInt();
+        return new UnCompressMaxMinInt();
 
-        case DATA_LONG:
+      case DATA_LONG:
 
-          return new UnCompressMaxMinLong();
+        return new UnCompressMaxMinLong();
 
-        case DATA_FLOAT:
+      case DATA_FLOAT:
 
-          return new UnCompressMaxMinFloat();
+        return new UnCompressMaxMinFloat();
 
-        default:
+      default:
 
-          return new UnCompressMaxMinDefault();
+        return new UnCompressMaxMinDefault();
 
-      }
     }
   }
 
@@ -881,6 +873,9 @@ public final class ValueCompressionUtil {
     DATA_LONG(), /**
      *
      */
+    DATA_BIGINT(), /**
+     *
+     */
     DATA_DOUBLE();
 
     /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractDistinctCountAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractDistinctCountAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractDistinctCountAggregatorObjectSet.java
deleted file mode 100644
index a02b1eb..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractDistinctCountAggregatorObjectSet.java
+++ /dev/null
@@ -1,103 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public abstract class AbstractDistinctCountAggregatorObjectSet implements MeasureAggregator {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  protected Set<Object> valueSetForObj;
-
-  public AbstractDistinctCountAggregatorObjectSet() {
-    valueSetForObj = new HashSet<Object>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-  }
-
-  /**
-   * Distinct count Aggregate function which update the Distinct count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSetForObj.add(newVal);
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    return null;
-  }
-
-  @Override public Double getDoubleValue() {
-    return (double) valueSetForObj.size();
-  }
-
-  @Override public Long getLongValue() {
-    return (long) valueSetForObj.size();
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return new BigDecimal(valueSetForObj.size());
-  }
-
-  @Override public Object getValueObject() {
-    return valueSetForObj.size();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    valueSetForObj.add(newValue);
-  }
-
-  @Override public boolean isFirstTime() {
-    return false;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-
-  }
-
-  public String toString() {
-    return valueSetForObj.size() + "";
-  }
-
-  @Override public void merge(byte[] value) {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMaxAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMaxAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMaxAggregator.java
deleted file mode 100644
index 13e6640..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMaxAggregator.java
+++ /dev/null
@@ -1,60 +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.carbondata.query.aggregator.impl;
-
-import java.io.ByteArrayInputStream;
-import java.io.ObjectInput;
-import java.io.ObjectInputStream;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.util.CarbonUtil;
-
-public abstract class AbstractMaxAggregator extends AbstractMeasureAggregatorMaxMin {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractMaxAggregator.class.getName());
-
-  protected void internalAgg(Object value) {
-    if (value instanceof Comparable) {
-      @SuppressWarnings("unchecked") Comparable<Object> newValue = ((Comparable<Object>) value);
-      aggVal = (aggVal == null || aggVal.compareTo(newValue) < 0) ? newValue : aggVal;
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteArrayInputStream bytesInputStream = null;
-    ObjectInput in = null;
-    try {
-      bytesInputStream = new ByteArrayInputStream(value);
-      in = new ObjectInputStream(bytesInputStream);
-      Object newVal = (Comparable<Object>) in.readObject();
-      internalAgg(newVal);
-      firstTime = false;
-    } catch (Exception e) {
-      LOGGER.error(e, "Problem while merging byte array in maxAggregator: " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(bytesInputStream);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorDummy.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorDummy.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorDummy.java
deleted file mode 100644
index 155687c..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorDummy.java
+++ /dev/null
@@ -1,69 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * AbstractMeasureAggregatorDummy
- * Used for custom Carbon Aggregator dummy
- */
-public abstract class AbstractMeasureAggregatorDummy extends AbstractMeasureAggregatorBasic {
-  private static final long serialVersionUID = 1L;
-
-  @Override public int compareTo(MeasureAggregator o) {
-    if (equals(o)) {
-      return 0;
-    }
-    return -1;
-  }
-
-  @Override public boolean equals(Object arg0) {
-    return super.equals(arg0);
-  }
-
-  @Override public int hashCode() {
-    return super.hashCode();
-  }
-
-  @Override public byte[] getByteArray() {
-    return null;
-  }
-
-  @Override public void merge(MeasureAggregator aggregator) {
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    return null;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-  }
-
-  @Override public void merge(byte[] value) {
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java
index 4f3163a..99a8ed9 100644
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java
@@ -34,6 +34,7 @@ import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.max.MaxAggregator;
 
 /**
  * AbstractMeasureAggregatorMaxMin
@@ -45,9 +46,9 @@ public abstract class AbstractMeasureAggregatorMaxMin implements MeasureAggregat
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(MaxAggregator.class.getName());
 
-  protected Comparable<Object> aggVal;
+  public Comparable<Object> aggVal;
 
-  protected boolean firstTime = true;
+  public boolean firstTime = true;
 
   protected abstract void internalAgg(Object value);
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMinAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMinAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMinAggregator.java
deleted file mode 100644
index 19fc69f..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMinAggregator.java
+++ /dev/null
@@ -1,60 +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.carbondata.query.aggregator.impl;
-
-import java.io.ByteArrayInputStream;
-import java.io.ObjectInput;
-import java.io.ObjectInputStream;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.util.CarbonUtil;
-
-public abstract class AbstractMinAggregator extends AbstractMeasureAggregatorMaxMin {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractMinAggregator.class.getName());
-
-  protected void internalAgg(Object value) {
-    if (value instanceof Comparable) {
-      @SuppressWarnings("unchecked") Comparable<Object> newValue = ((Comparable<Object>) value);
-      aggVal = (aggVal == null || aggVal.compareTo(newValue) > 0) ? newValue : aggVal;
-    }
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteArrayInputStream bis = null;
-    ObjectInput objectInput = null;
-    try {
-      bis = new ByteArrayInputStream(value);
-      objectInput = new ObjectInputStream(bis);
-      Object newVal = (Comparable<Object>) objectInput.readObject();
-      internalAgg(newVal);
-      firstTime = false;
-    } catch (Exception e) {
-      LOGGER.error(e, "Problem while merging byte array in minAggregator: " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(bis);
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AvgBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AvgBigDecimalAggregator.java
deleted file mode 100644
index 8c67cfc..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgBigDecimalAggregator.java
+++ /dev/null
@@ -1,214 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class AvgBigDecimalAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 5463736686281089871L;
-
-  /**
-   * total number of aggregate values
-   */
-  protected double count;
-
-  /**
-   * aggregate value
-   */
-  protected BigDecimal aggVal;
-
-  public AvgBigDecimalAggregator() {
-    aggVal = new BigDecimal(0);
-  }
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        byte[] valueByte = new byte[buffer.getInt()];
-        buffer.get(valueByte);
-        BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-        aggVal = aggVal.add(valueBigDecimal);
-
-        count += buffer.getDouble();
-        firstTime = false;
-      }
-      return;
-    }
-
-    if (firstTime) {
-      aggVal = (BigDecimal) newVal;
-      firstTime = false;
-    } else {
-      aggVal = aggVal.add((BigDecimal) newVal);
-    }
-    count++;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      BigDecimal value = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-      aggVal = aggVal.add(value);
-      firstTime = false;
-      count++;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    byte[] bytes = DataTypeUtil.bigDecimalToByte(aggVal);
-    ByteBuffer allocate =
-        ByteBuffer.allocate(4 + bytes.length + CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    allocate.putInt(bytes.length);
-    allocate.put(bytes);
-    allocate.putDouble(count);
-    allocate.rewind();
-
-    return allocate.array();
-  }
-
-  /**
-   * Return the average of the aggregate values
-   *
-   * @return average aggregate value
-   */
-  @Override public BigDecimal getBigDecimalValue() {
-    return aggVal.divide(new BigDecimal(count), 6);
-  }
-
-  /**
-   * This method merge the aggregated value, in average aggregator it will add
-   * count and aggregate value
-   *
-   * @param aggregator Avg Aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    AvgBigDecimalAggregator avgAggregator = (AvgBigDecimalAggregator) aggregator;
-    if (!avgAggregator.isFirstTime()) {
-      aggVal = aggVal.add(avgAggregator.aggVal);
-      count += avgAggregator.count;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * This method return the average value as an object
-   *
-   * @return average value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal.divide(new BigDecimal(count));
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (BigDecimal) newValue;
-    count = 1;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeUTF(aggVal.toString());
-    output.writeDouble(count);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = new BigDecimal(inPut.readUTF());
-    count = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    AvgBigDecimalAggregator avg = new AvgBigDecimalAggregator();
-    avg.aggVal = aggVal;
-    avg.count = count;
-    avg.firstTime = firstTime;
-    return avg;
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    BigDecimal val = getBigDecimalValue();
-    BigDecimal otherVal = o.getBigDecimalValue();
-
-    return val.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof AvgBigDecimalAggregator)) {
-      return false;
-    }
-    AvgBigDecimalAggregator o = (AvgBigDecimalAggregator) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-
-    byte[] valueByte = new byte[buffer.getInt()];
-    buffer.get(valueByte);
-    BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-    aggVal = aggVal.add(valueBigDecimal);
-    count += buffer.getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return (aggVal.divide(new BigDecimal(count))) + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new AvgBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AvgDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AvgDoubleAggregator.java
deleted file mode 100644
index aea7007..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgDoubleAggregator.java
+++ /dev/null
@@ -1,206 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class AvgDoubleAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 5463736686281089871L;
-
-  /**
-   * total number of aggregate values
-   */
-  protected double count;
-
-  /**
-   * aggregate value
-   */
-  protected double aggVal;
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(double newVal) {
-    aggVal += newVal;
-    count++;
-    firstTime = false;
-  }
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        aggVal += buffer.getDouble();
-        count += buffer.getDouble();
-        firstTime = false;
-      }
-      return;
-    }
-    aggVal += ((Number) newVal).doubleValue();
-    count++;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal += dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      count++;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(2 * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putDouble(aggVal);
-    buffer.putDouble(count);
-    return buffer.array();
-  }
-
-  /**
-   * Return the average of the aggregate values
-   *
-   * @return average aggregate value
-   */
-  @Override public Double getDoubleValue() {
-    return aggVal / count;
-  }
-
-  /**
-   * This method merge the aggregated value, in average aggregator it will add
-   * count and aggregate value
-   *
-   * @param aggregator Avg Aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    AvgDoubleAggregator avgAggregator = (AvgDoubleAggregator) aggregator;
-    if (!avgAggregator.isFirstTime()) {
-      aggVal += avgAggregator.aggVal;
-      count += avgAggregator.count;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * This method return the average value as an object
-   *
-   * @return average value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal / count;
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Double) newValue;
-    count = 1;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeDouble(aggVal);
-    output.writeDouble(count);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readDouble();
-    count = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    AvgDoubleAggregator avg = new AvgDoubleAggregator();
-    avg.aggVal = aggVal;
-    avg.count = count;
-    avg.firstTime = firstTime;
-    return avg;
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    double val = getDoubleValue();
-    double otherVal = o.getDoubleValue();
-    if (val > otherVal) {
-      return 1;
-    }
-    if (val < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof AvgDoubleAggregator)) {
-      return false;
-    }
-    AvgDoubleAggregator o = (AvgDoubleAggregator)obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    aggVal += buffer.getDouble();
-    count += buffer.getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return (aggVal / count) + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new AvgDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/AvgLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AvgLongAggregator.java
deleted file mode 100644
index d608c90..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AvgLongAggregator.java
+++ /dev/null
@@ -1,191 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class AvgLongAggregator extends AbstractMeasureAggregatorBasic {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 5463736686281089871L;
-
-  /**
-   * total number of aggregate values
-   */
-  protected double count;
-
-  /**
-   * aggregate value
-   */
-  protected long aggVal;
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        aggVal += buffer.getLong();
-        count += buffer.getDouble();
-        firstTime = false;
-      }
-      return;
-    }
-    aggVal += (Long) newVal;
-    count++;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal += dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-      count++;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(
-        CarbonCommonConstants.LONG_SIZE_IN_BYTE + CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putLong(aggVal);
-    buffer.putDouble(count);
-    return buffer.array();
-  }
-
-  @Override public Long getLongValue() {
-    return aggVal / (long) count;
-  }
-
-  /**
-   * This method merge the aggregated value, in average aggregator it will add
-   * count and aggregate value
-   *
-   * @param aggregator Avg Aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    AvgLongAggregator avgAggregator = (AvgLongAggregator) aggregator;
-    if (!avgAggregator.isFirstTime()) {
-      aggVal += avgAggregator.aggVal;
-      count += avgAggregator.count;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * This method return the average value as an object
-   *
-   * @return average value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal / count;
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Long) newValue;
-    count = 1;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeLong(aggVal);
-    output.writeDouble(count);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readLong();
-    count = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    AvgLongAggregator avg = new AvgLongAggregator();
-    avg.aggVal = aggVal;
-    avg.count = count;
-    avg.firstTime = firstTime;
-    return avg;
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    long val = getLongValue();
-    long otherVal = o.getLongValue();
-    if (val > otherVal) {
-      return 1;
-    } else if (val < otherVal) {
-      return -1;
-    } else {
-      return 0;
-    }
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof AvgLongAggregator)) {
-      return false;
-    }
-    AvgLongAggregator o = (AvgLongAggregator)obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    aggVal += buffer.getLong();
-    count += buffer.getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return (aggVal / count) + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new AvgLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java
deleted file mode 100644
index 12c2061..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/CountAggregator.java
+++ /dev/null
@@ -1,204 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return total count of values
- */
-public class CountAggregator implements MeasureAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 2678878935295306313L;
-
-  /**
-   * aggregate value
-   */
-  private double aggVal;
-
-  /**
-   * Count Aggregate function which update the total count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(double newVal) {
-    aggVal++;
-  }
-
-  /**
-   * Count Aggregate function which update the total count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    aggVal++;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal++;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putDouble(aggVal);
-    return buffer.array();
-  }
-
-  /**
-   * Returns the total count
-   *
-   * @return total count
-   */
-  @Override public Double getDoubleValue() {
-    return aggVal;
-  }
-
-  @Override public Long getLongValue() {
-    return (long) aggVal;
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return new BigDecimal(aggVal);
-  }
-
-  /**
-   * Merge the total count with the aggregator
-   *
-   * @param aggregator count aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    CountAggregator countAggregator = (CountAggregator) aggregator;
-    aggVal += countAggregator.aggVal;
-  }
-
-  /**
-   * Overloaded Aggregate function will be used for Aggregate tables because
-   * aggregate table will have fact_count as a measure. It will update the
-   * total count
-   *
-   * @param newVal
-   *            new value
-   * @param factCount
-   *            total fact count
-   *
-   */
-  //    @Override
-  //    public void agg(double newVal, double factCount)
-  //    {
-  //        agg(newVal, null, 0, 0);
-  //    }
-
-  /**
-   * This method return the count value as an object
-   *
-   * @return count value as an object
-   */
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal += Double.parseDouble(String.valueOf(newValue));
-  }
-
-  @Override public boolean isFirstTime() {
-    return false;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeDouble(aggVal);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    aggVal = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    CountAggregator aggregator = new CountAggregator();
-    aggregator.aggVal = aggVal;
-    return aggregator;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    aggVal += buffer.getDouble();
-  }
-
-  @Override public int compareTo(MeasureAggregator obj) {
-    double val = getDoubleValue();
-    double otherVal = obj.getDoubleValue();
-    if (val > otherVal) {
-      return 1;
-    }
-    if (val < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof CountAggregator)) {
-      return false;
-    }
-    CountAggregator o = (CountAggregator)obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new CountAggregator();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregator.java
deleted file mode 100644
index f634ab3..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregator.java
+++ /dev/null
@@ -1,319 +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.carbondata.query.aggregator.impl;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-import org.roaringbitmap.IntIterator;
-import org.roaringbitmap.RoaringBitmap;
-
-/**
- * * The distinct count aggregator
- * Ex:
- * ID NAME Sales
- * <p>1 a 200
- * <p>2 a 100
- * <p>3 a 200
- * select count(distinct sales) # would result 2
- * select count(sales) # would result 3
- */
-public class DistinctCountAggregator implements MeasureAggregator {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DistinctCountAggregator.class.getName());
-  /**
-   *
-   */
-  private static final long serialVersionUID = 6313463368629960186L;
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten
-   * the Aggregators. There is no aggregation expected after setting this value.
-   */
-  private Double computedFixedValue;
-  /**
-   *
-   */
-  //    private Set<Double> valueSet;
-  private RoaringBitmap valueSet;
-
-  private byte[] data;
-
-  private double minValue;
-
-  public DistinctCountAggregator(Object minValue) {
-    valueSet = new RoaringBitmap();
-    if (minValue instanceof BigDecimal) {
-      this.minValue = ((BigDecimal) minValue).doubleValue();
-    } else if (minValue instanceof Long) {
-      this.minValue = ((Long) minValue).doubleValue();
-    } else {
-      this.minValue = (Double) minValue;
-    }
-  }
-
-  public DistinctCountAggregator() {
-    valueSet = new RoaringBitmap();
-  }
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-    valueSet.add((int) (newVal - minValue));
-  }
-
-  /**
-   * Distinct count Aggregate function which update the Distinct count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      byte[] values = (byte[]) newVal;
-      ByteBuffer buffer = ByteBuffer.wrap(values);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        valueSet.add(buffer.getInt());
-      }
-      return;
-    } else {
-      double value = new Double(newVal.toString());
-      agg(value);
-    }
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add((int) dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (valueSet.getCardinality() == 0) {
-      return new byte[0];
-    }
-    IntIterator iterator = valueSet.getIntIterator();
-    ByteBuffer buffer = ByteBuffer.allocate(valueSet.getCardinality() * 4 + 8);
-    buffer.putDouble(minValue);
-    while (iterator.hasNext()) {
-      buffer.putInt(iterator.next());
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(RoaringBitmap set2, double minValue) {
-    if (this.minValue == minValue) {
-      valueSet.or(set2);
-    } else {
-      if (this.minValue > minValue) {
-        IntIterator intIterator = valueSet.getIntIterator();
-        while (intIterator.hasNext()) {
-          set2.add((int) ((double) (intIterator.next() + this.minValue) - minValue));
-        }
-        this.minValue = minValue;
-        this.valueSet = set2;
-      } else {
-        IntIterator intIterator = set2.getIntIterator();
-        while (intIterator.hasNext()) {
-          valueSet.add((int) ((double) (intIterator.next() + minValue) - this.minValue));
-        }
-      }
-    }
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountAggregator distinctCountAggregator = (DistinctCountAggregator) aggregator;
-    readData();
-    distinctCountAggregator.readData();
-    if (distinctCountAggregator.valueSet != null) {
-      agg(distinctCountAggregator.valueSet, distinctCountAggregator.minValue);
-    }
-  }
-
-  @Override public Double getDoubleValue() {
-    if (computedFixedValue == null) {
-      readData();
-      return (double) valueSet.getCardinality();
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Long getLongValue() {
-    if (computedFixedValue == null) {
-      readData();
-      return (long) valueSet.getCardinality();
-    }
-    return computedFixedValue.longValue();
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    if (computedFixedValue == null) {
-      readData();
-      return new BigDecimal(valueSet.getCardinality());
-    }
-    return new BigDecimal(computedFixedValue);
-  }
-
-  @Override public Object getValueObject() {
-    return valueSet.getCardinality();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (Double) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return false;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-
-    if (computedFixedValue != null) {
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
-      byteBuffer.putInt(-1);
-      byteBuffer.putDouble(computedFixedValue);
-      byteBuffer.flip();
-      output.write(byteBuffer.array());
-    } else {
-      if (valueSet != null) {
-        valueSet.serialize(output);
-      } else {
-        output.write(data);
-      }
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    valueSet = new RoaringBitmap();
-    valueSet.deserialize(inPut);
-  }
-
-  private void readData() {
-    if (data != null && (valueSet == null || valueSet.isEmpty())) {
-      ByteArrayInputStream stream = new ByteArrayInputStream(data);
-      DataInputStream outputStream = new DataInputStream(stream);
-      try {
-        readData(outputStream);
-        outputStream.close();
-        data = null;
-      } catch (IOException e) {
-        LOGGER.error(e, e.getMessage());
-      }
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountAggregator aggr = new DistinctCountAggregator(minValue);
-    aggr.valueSet = valueSet.clone();
-    return aggr;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    double compFixedVal = getDoubleValue();
-    double otherVal = measureAggr.getDoubleValue();
-    if (compFixedVal > otherVal) {
-      return 1;
-    }
-    if (compFixedVal < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof DistinctCountAggregator)) {
-      return false;
-    }
-    DistinctCountAggregator o = (DistinctCountAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-    DataOutputStream outputStream = new DataOutputStream(byteStream);
-    try {
-      writeData(outputStream);
-    } catch (IOException ex) {
-      LOGGER.error(ex, ex.getMessage());
-    }
-    data = byteStream.toByteArray();
-    valueSet = null;
-    return this;
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      readData();
-      return valueSet.getCardinality() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  public RoaringBitmap getBitMap() {
-    return valueSet;
-  }
-
-  public double getMinValue() {
-    return minValue;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    double currentMinValue = buffer.getDouble();
-    while (buffer.hasRemaining()) {
-      agg(buffer.getInt() + currentMinValue);
-    }
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new DistinctCountAggregator();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregatorObjectSet.java
deleted file mode 100644
index f6210dc..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountAggregatorObjectSet.java
+++ /dev/null
@@ -1,96 +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.carbondata.query.aggregator.impl;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctCountAggregatorObjectSet extends AbstractDistinctCountAggregatorObjectSet {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-    valueSetForObj.add(newVal);
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-    }
-  }
-
-  private void agg(Set<Object> set2) {
-    valueSetForObj.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountAggregatorObjectSet distinctCountAggregator =
-        (DistinctCountAggregatorObjectSet) aggregator;
-    agg(distinctCountAggregator.valueSetForObj);
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountAggregatorObjectSet aggregator = new DistinctCountAggregatorObjectSet();
-    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    double valueSetForObjSize = getDoubleValue();
-    double otherVal = measureAggr.getDoubleValue();
-    if (valueSetForObjSize > otherVal) {
-      return 1;
-    }
-    if (valueSetForObjSize < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof DistinctCountAggregatorObjectSet)) {
-      return false;
-    }
-    DistinctCountAggregatorObjectSet o = (DistinctCountAggregatorObjectSet) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DistinctCountAggregatorObjectSet();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountBigDecimalAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountBigDecimalAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountBigDecimalAggregatorObjectSet.java
deleted file mode 100644
index 21dc142..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountBigDecimalAggregatorObjectSet.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.carbondata.query.aggregator.impl;
-
-import java.math.BigDecimal;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctCountBigDecimalAggregatorObjectSet
-    extends AbstractDistinctCountAggregatorObjectSet {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index));
-    }
-  }
-
-  private void agg(Set<Object> set2) {
-    valueSetForObj.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountBigDecimalAggregatorObjectSet distinctCountBigDecimalAggregatorObjectSet =
-        (DistinctCountBigDecimalAggregatorObjectSet) aggregator;
-    agg(distinctCountBigDecimalAggregatorObjectSet.valueSetForObj);
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountBigDecimalAggregatorObjectSet aggregator =
-        new DistinctCountBigDecimalAggregatorObjectSet();
-    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    BigDecimal valueSetForObjSize = getBigDecimalValue();
-    BigDecimal otherVal = measureAggr.getBigDecimalValue();
-    return valueSetForObjSize.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof DistinctCountBigDecimalAggregatorObjectSet)) {
-      return false;
-    }
-    DistinctCountBigDecimalAggregatorObjectSet o = (DistinctCountBigDecimalAggregatorObjectSet) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DistinctCountBigDecimalAggregatorObjectSet();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountLongAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountLongAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountLongAggregatorObjectSet.java
deleted file mode 100644
index 7f8cd0a..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctCountLongAggregatorObjectSet.java
+++ /dev/null
@@ -1,89 +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.carbondata.query.aggregator.impl;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctCountLongAggregatorObjectSet extends AbstractDistinctCountAggregatorObjectSet {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index));
-    }
-  }
-
-  private void agg(Set<Object> set2) {
-    valueSetForObj.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountLongAggregatorObjectSet distinctCountAggregator =
-        (DistinctCountLongAggregatorObjectSet) aggregator;
-    agg(distinctCountAggregator.valueSetForObj);
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountLongAggregatorObjectSet aggregator = new DistinctCountLongAggregatorObjectSet();
-    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    long valueSetForObjSize = getLongValue();
-    long otherVal = measureAggr.getLongValue();
-    if (valueSetForObjSize > otherVal) {
-      return 1;
-    }
-    if (valueSetForObjSize < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof DistinctCountLongAggregatorObjectSet)) {
-      return false;
-    }
-    DistinctCountLongAggregatorObjectSet o = (DistinctCountLongAggregatorObjectSet) obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DistinctCountLongAggregatorObjectSet();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctStringCountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctStringCountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctStringCountAggregator.java
deleted file mode 100644
index 89cdeeb..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DistinctStringCountAggregator.java
+++ /dev/null
@@ -1,165 +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.carbondata.query.aggregator.impl;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-import java.nio.charset.Charset;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctStringCountAggregator implements MeasureAggregator {
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  private Set<String> valueSetForStr;
-
-  public DistinctStringCountAggregator() {
-    this.valueSetForStr = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  public void agg(double newVal) {
-  }
-
-  public void agg(String newVal) {
-    this.valueSetForStr.add(newVal);
-  }
-
-  private void agg(Set<String> set2) {
-    this.valueSetForStr.addAll(set2);
-  }
-
-  public void merge(MeasureAggregator aggregator) {
-    DistinctStringCountAggregator distinctCountAggregator =
-        (DistinctStringCountAggregator) aggregator;
-    agg(distinctCountAggregator.valueSetForStr);
-  }
-
-  public Double getDoubleValue() {
-    return (double) this.valueSetForStr.size();
-  }
-
-  public Long getLongValue() {
-    return (long) this.valueSetForStr.size();
-  }
-
-  public BigDecimal getBigDecimalValue() {
-    return new BigDecimal(this.valueSetForStr.size());
-  }
-
-  public Object getValueObject() {
-    return Integer.valueOf(this.valueSetForStr.size());
-  }
-
-  public void setNewValue(Object newValue) {
-  }
-
-  public boolean isFirstTime() {
-    return false;
-  }
-
-  public void writeData(DataOutput output) throws IOException {
-    int length = this.valueSetForStr.size() * 8;
-    ByteBuffer byteBuffer = ByteBuffer.allocate(length + 4);
-    byteBuffer.putInt(length);
-    for (String val : this.valueSetForStr) {
-      byte[] b = val.getBytes(Charset.defaultCharset());
-      byteBuffer.putInt(b.length);
-      byteBuffer.put(b);
-    }
-    byteBuffer.flip();
-    output.write(byteBuffer.array());
-  }
-
-  public void readData(DataInput inPut) throws IOException {
-    int length = inPut.readInt();
-    length /= 8;
-    this.valueSetForStr = new HashSet<String>(length + 1, 1.0F);
-    for (int i = 0; i < length; i++) {
-      byte[] b = new byte[inPut.readInt()];
-      inPut.readFully(b);
-      this.valueSetForStr.add(new String(b, Charset.defaultCharset()));
-    }
-  }
-
-  public MeasureAggregator getCopy() {
-    DistinctStringCountAggregator aggregator = new DistinctStringCountAggregator();
-    aggregator.valueSetForStr = new HashSet<String>(this.valueSetForStr);
-    return aggregator;
-  }
-
-  public int compareTo(MeasureAggregator o) {
-    double val = getDoubleValue();
-    double otherVal = o.getDoubleValue();
-    if (val > otherVal) {
-      return 1;
-    }
-    if (val < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof DistinctStringCountAggregator)) {
-      return false;
-    }
-    DistinctStringCountAggregator o = (DistinctStringCountAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public void agg(Object newVal) {
-    this.valueSetForStr.add((String) newVal);
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-  }
-
-  @Override public byte[] getByteArray() {
-    return null;
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  public String toString() {
-    return valueSetForStr.size() + "";
-  }
-
-  @Override public void merge(byte[] value) {
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new DistinctStringCountAggregator();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/DummyBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/DummyBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/DummyBigDecimalAggregator.java
deleted file mode 100644
index 9437f14..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/DummyBigDecimalAggregator.java
+++ /dev/null
@@ -1,63 +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.carbondata.query.aggregator.impl;
-
-import java.math.BigDecimal;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DummyBigDecimalAggregator extends AbstractMeasureAggregatorDummy {
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * aggregate value
-   */
-  private BigDecimal aggVal;
-
-  @Override public void agg(Object newVal) {
-    aggVal = (BigDecimal) newVal;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-      firstTime = false;
-    }
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return aggVal;
-  }
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (BigDecimal) newValue;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new DummyBigDecimalAggregator();
-  }
-}


[23/56] [abbrv] incubator-carbondata git commit: Optimized detail query flow and cleanup (#691)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
index 09fa50c..d2aa014 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
@@ -18,7 +18,6 @@
  */
 package org.carbondata.query.carbon.result.iterator;
 
-import java.util.Arrays;
 import java.util.List;
 
 import org.carbondata.common.logging.LogService;
@@ -27,12 +26,15 @@ import org.carbondata.core.carbon.datastore.DataRefNode;
 import org.carbondata.core.carbon.datastore.DataRefNodeFinder;
 import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
+import org.carbondata.query.carbon.processor.AbstractDataBlockIterator;
+import org.carbondata.query.carbon.processor.impl.DataBlockIteratorImpl;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
@@ -60,101 +62,73 @@ public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
   /**
    * number of cores which can be used
    */
-  private long numberOfCores;
+  private int batchSize;
 
   /**
-   * keep track of number of blocklet per block
+   * file reader which will be used to execute the query
    */
-  private long[] totalNumberBlockletPerSlice;
+  protected FileHolder fileReader;
 
-  /**
-   * total number of blocklet to be executed
-   */
-  private long totalNumberOfNode;
+  protected AbstractDataBlockIterator dataBlockIterator;
 
-  /**
-   * current counter to check how blocklet has been executed
-   */
-  protected long currentCounter;
+  protected boolean nextBatch = false;
 
-  /**
-   * keep the track of number of blocklet of a block has been executed
-   */
-  private long[] numberOfBlockletExecutedPerBlock;
-
-  /**
-   * block index to be executed
-   */
-  protected int[] blockIndexToBeExecuted;
-
-  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos,
-      QueryExecutorProperties executerProperties, QueryModel queryModel,
-      InternalQueryExecutor queryExecutor) {
-    int recordSize = 0;
-    String defaultInMemoryRecordsSize =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.INMEMORY_REOCRD_SIZE);
-    if (null != defaultInMemoryRecordsSize) {
+  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
+    String batchSizeString =
+        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE);
+    if (null != batchSizeString) {
       try {
-        recordSize = Integer.parseInt(defaultInMemoryRecordsSize);
+        batchSize = Integer.parseInt(batchSizeString);
       } catch (NumberFormatException ne) {
         LOGGER.error("Invalid inmemory records size. Using default value");
-        recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
+        batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
       }
+    } else {
+      batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
     }
-    this.numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    if (numberOfCores == 0) {
-      numberOfCores++;
-    }
-    executor = queryExecutor;
+
     this.blockExecutionInfos = infos;
-    this.blockIndexToBeExecuted = new int[(int) numberOfCores];
+    this.fileReader = FileFactory.getFileHolder(
+        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
     intialiseInfos();
   }
 
   private void intialiseInfos() {
-    this.totalNumberBlockletPerSlice = new long[blockExecutionInfos.size()];
-    this.numberOfBlockletExecutedPerBlock = new long[blockExecutionInfos.size()];
-    int index = -1;
     for (BlockExecutionInfo blockInfo : blockExecutionInfos) {
-      ++index;
       DataRefNodeFinder finder = new BTreeDataRefNodeFinder(blockInfo.getEachColumnValueSize());
       DataRefNode startDataBlock = finder
           .findFirstDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getStartKey());
       DataRefNode endDataBlock = finder
           .findLastDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getEndKey());
-
-      this.totalNumberBlockletPerSlice[index] =
-          endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
-      totalNumberOfNode += this.totalNumberBlockletPerSlice[index];
+      long numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
       blockInfo.setFirstDataBlock(startDataBlock);
-      blockInfo.setNumberOfBlockToScan(1);
+      blockInfo.setNumberOfBlockToScan(numberOfBlockToScan);
     }
-
   }
 
   @Override public boolean hasNext() {
-    return currentCounter < totalNumberOfNode;
+    if ((dataBlockIterator != null && dataBlockIterator.hasNext()) || nextBatch) {
+      return true;
+    } else {
+      dataBlockIterator = getDataBlockIterator();
+      while (dataBlockIterator != null) {
+        if (dataBlockIterator.hasNext()) {
+          return true;
+        }
+        dataBlockIterator = getDataBlockIterator();
+      }
+      return false;
+    }
   }
 
-  protected int updateSliceIndexToBeExecuted() {
-    Arrays.fill(blockIndexToBeExecuted, -1);
-    int currentSliceIndex = 0;
-    int i = 0;
-    for (; i < (int) numberOfCores; ) {
-      if (this.totalNumberBlockletPerSlice[currentSliceIndex]
-          > this.numberOfBlockletExecutedPerBlock[currentSliceIndex]) {
-        this.numberOfBlockletExecutedPerBlock[currentSliceIndex]++;
-        blockIndexToBeExecuted[i] = currentSliceIndex;
-        i++;
-      }
-      currentSliceIndex++;
-      if (currentSliceIndex >= totalNumberBlockletPerSlice.length) {
-        break;
-      }
+  private DataBlockIteratorImpl getDataBlockIterator() {
+    if(blockExecutionInfos.size() > 0) {
+      BlockExecutionInfo executionInfo = blockExecutionInfos.get(0);
+      blockExecutionInfos.remove(executionInfo);
+      return new DataBlockIteratorImpl(executionInfo, fileReader, batchSize);
     }
-    return i;
+    return null;
   }
 
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
index 3641e75..458a4c8 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
@@ -19,20 +19,17 @@
 package org.carbondata.query.carbon.result.iterator;
 
 import java.util.List;
+import java.util.concurrent.Callable;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
 
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.model.QueryModel;
 import org.carbondata.query.carbon.result.BatchResult;
 import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPreparatorImpl;
 
 /**
  * In case of detail query we cannot keep all the records in memory so for
@@ -42,48 +39,46 @@ import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPrepa
 public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator {
 
   /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DetailQueryResultIterator.class.getName());
-
-  /**
    * to prepare the result
    */
   private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
 
-  public DetailQueryResultIterator(List<BlockExecutionInfo> infos,
-      QueryExecutorProperties executerProperties, QueryModel queryModel,
-      InternalQueryExecutor queryExecutor) {
-    super(infos, executerProperties, queryModel, queryExecutor);
-    this.queryResultPreparator =
-        new DetailQueryResultPreparatorImpl(executerProperties, queryModel);
+  private ExecutorService execService = Executors.newFixedThreadPool(1);
+
+  private Future<BatchResult> future;
+
+  public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
+      QueryResultPreparator queryResultPreparator) {
+    super(infos, queryModel);
+    this.queryResultPreparator = queryResultPreparator;
   }
 
   @Override public BatchResult next() {
-    currentCounter += updateSliceIndexToBeExecuted();
-    CarbonIterator<Result> result = null;
+    BatchResult result;
     try {
-      result = executor.executeQuery(blockExecutionInfos, blockIndexToBeExecuted);
-    } catch (QueryExecutionException e) {
-      throw new RuntimeException(e.getCause().getMessage());
-    }
-    for (int i = 0; i < blockIndexToBeExecuted.length; i++) {
-      if (blockIndexToBeExecuted[i] != -1) {
-        blockExecutionInfos.get(blockIndexToBeExecuted[i]).setFirstDataBlock(
-            blockExecutionInfos.get(blockIndexToBeExecuted[i]).getFirstDataBlock()
-                .getNextDataRefNode());
+      if (future == null) {
+        future = execute();
       }
-    }
-    if (null != result) {
-      Result next = result.next();
-      if (next.size() > 0) {
-        return queryResultPreparator.prepareQueryResult(next);
+      result = future.get();
+      nextBatch = false;
+      if (hasNext()) {
+        nextBatch = true;
+        future = execute();
       } else {
-        return new BatchResult();
+        fileReader.finish();
       }
-    } else {
-      return new BatchResult();
+    } catch (Exception ex) {
+      fileReader.finish();
+      throw new RuntimeException(ex.getCause().getMessage());
     }
+    return result;
+  }
+
+  private Future<BatchResult> execute() {
+    return execService.submit(new Callable<BatchResult>() {
+      @Override public BatchResult call() throws QueryExecutionException {
+        return queryResultPreparator.prepareQueryResult(dataBlockIterator.next());
+      }
+    });
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
deleted file mode 100644
index 2b14793..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailRawQueryResultIterator.java
+++ /dev/null
@@ -1,118 +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.carbondata.query.carbon.result.iterator;
-
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.result.preparator.impl.RawQueryResultPreparatorImpl;
-
-/**
- * In case of detail query we cannot keep all the records in memory so for
- * executing that query are returning a iterator over block and every time next
- * call will come it will execute the block and return the result
- */
-public class DetailRawQueryResultIterator extends AbstractDetailQueryResultIterator {
-
-  private ExecutorService execService = Executors.newFixedThreadPool(1);
-
-  private Future<ResultInfo> future;
-
-  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
-
-  public DetailRawQueryResultIterator(List<BlockExecutionInfo> infos,
-      QueryExecutorProperties executerProperties, QueryModel queryModel,
-      InternalQueryExecutor queryExecutor) {
-    super(infos, executerProperties, queryModel, queryExecutor);
-    this.queryResultPreparator = new RawQueryResultPreparatorImpl(executerProperties, queryModel);
-  }
-
-  @Override public BatchResult next() {
-    BatchResult result;
-    if (future == null) {
-      future = execute();
-    }
-    ResultInfo resultFromFuture = getResultFromFuture(future);
-    result = resultFromFuture.result;
-    currentCounter += resultFromFuture.counter;
-    if (hasNext()) {
-      future = execute();
-    }
-    return result;
-  }
-
-  private ResultInfo getResultFromFuture(Future<ResultInfo> future) {
-    try {
-      return future.get();
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-    return new ResultInfo();
-  }
-
-  private Future<ResultInfo> execute() {
-    return execService.submit(new Callable<ResultInfo>() {
-      @Override public ResultInfo call() {
-        CarbonIterator<Result> result = null;
-        int counter =  updateSliceIndexToBeExecuted();
-        try {
-          result = executor.executeQuery(blockExecutionInfos, blockIndexToBeExecuted);
-        } catch (QueryExecutionException ex) {
-          throw new RuntimeException(ex.getCause());
-        }
-        for (int i = 0; i < blockIndexToBeExecuted.length; i++) {
-          if (blockIndexToBeExecuted[i] != -1) {
-            blockExecutionInfos.get(blockIndexToBeExecuted[i]).setFirstDataBlock(
-                blockExecutionInfos.get(blockIndexToBeExecuted[i]).getFirstDataBlock()
-                    .getNextDataRefNode());
-          }
-        }
-        BatchResult batchResult;
-        if (null != result) {
-          Result next = result.next();
-          batchResult = queryResultPreparator.prepareQueryResult(next);
-        } else {
-          batchResult = queryResultPreparator.prepareQueryResult(null);
-        }
-        ResultInfo resultInfo = new ResultInfo();
-        resultInfo.counter = counter;
-        resultInfo.result = batchResult;
-        return resultInfo;
-      }
-    });
-  }
-
-  private static class ResultInfo {
-    private int counter;
-    private BatchResult result;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
index 0ae6651..a6a8fbc 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
@@ -4,6 +4,9 @@ import java.util.List;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QueryMeasure;
@@ -13,6 +16,8 @@ import org.carbondata.query.carbon.result.BatchRawResult;
 import org.carbondata.query.carbon.result.BatchResult;
 import org.carbondata.query.carbon.result.ListBasedResultWrapper;
 import org.carbondata.query.carbon.result.Result;
+import org.carbondata.query.carbon.util.DataTypeUtil;
+import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 
 /**
  * It does not decode the dictionary.
@@ -54,29 +59,67 @@ public class RawQueryResultPreparatorImpl
   @Override public BatchResult prepareQueryResult(
       Result<List<ListBasedResultWrapper>, Object> scannedResult) {
     if ((null == scannedResult || scannedResult.size() < 1)) {
-      BatchRawResult batchRawResult = new BatchRawResult();
-      batchRawResult.setQuerySchemaInfo(querySchemaInfo);
-      return batchRawResult;
+      return new BatchRawResult();
     }
+    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
     int msrSize = queryExecuterProperties.measureDataTypes.length;
+    int dimSize = queryDimensions.length;
+    int[] order = querySchemaInfo.getQueryReverseOrder();
     Object[][] resultData = new Object[scannedResult.size()][];
     Object[] value;
     Object[] row;
     int counter = 0;
-    while (scannedResult.hasNext()) {
-      value = scannedResult.getValue();
-      row = new Object[msrSize + 1];
-      row[0] = scannedResult.getKey();
-      if(value != null) {
-        System.arraycopy(value, 0, row, 1, msrSize);
+    if (queryModel.isRawBytesDetailQuery()) {
+      while (scannedResult.hasNext()) {
+        value = scannedResult.getValue();
+        row = new Object[msrSize + 1];
+        row[0] = scannedResult.getKey();
+        if (value != null) {
+          assert (value.length == msrSize);
+          System.arraycopy(value, 0, row, 1, msrSize);
+        }
+        resultData[counter] = row;
+        counter++;
+      }
+    } else {
+      while (scannedResult.hasNext()) {
+        value = scannedResult.getValue();
+        row = new Object[msrSize + dimSize];
+        ByteArrayWrapper key = scannedResult.getKey();
+        if (key != null) {
+          long[] surrogateResult = querySchemaInfo.getKeyGenerator()
+              .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
+          int noDictionaryColumnIndex = 0;
+          for (int i = 0; i < dimSize; i++) {
+            if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
+              row[order[i]] = DataTypeUtil.getDataBasedOnDataType(
+                  new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
+                  queryDimensions[i].getDimension().getDataType());
+            } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+              DirectDictionaryGenerator directDictionaryGenerator =
+                  DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
+                      queryDimensions[i].getDimension().getDataType());
+              if (directDictionaryGenerator != null) {
+                row[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
+                    (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
+              }
+            } else {
+              row[order[i]] =
+                  (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
+            }
+          }
+        }
+        for (int i = 0; i < msrSize; i++) {
+          row[order[i + queryDimensions.length]] = value[i];
+        }
+        resultData[counter] = row;
+        counter++;
       }
-      resultData[counter] = row;
-      counter ++;
     }
+
     LOGGER.info("###########################---- Total Number of records" + scannedResult.size());
     BatchRawResult result = new BatchRawResult();
     result.setRows(resultData);
-    result.setQuerySchemaInfo(querySchemaInfo);
     return result;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java b/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
index 5485f99..2a7c5ae 100644
--- a/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
+++ b/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
@@ -29,6 +29,8 @@ import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.util.CarbonProperties;
 
+import org.apache.spark.unsafe.types.UTF8String;
+
 /**
  * Utility for data type
  */
@@ -95,7 +97,7 @@ public class DataTypeUtil {
               new org.apache.spark.sql.types.Decimal();
           return decConverter.set(scalaDecVal);
         default:
-          return data;
+          return UTF8String.fromString(data);
       }
     } catch (NumberFormatException ex) {
       LOGGER.error("Problem while converting data type" + data);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java b/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java
deleted file mode 100644
index a733ec6..0000000
--- a/core/src/main/java/org/carbondata/query/merger/exception/ResultMergerException.java
+++ /dev/null
@@ -1,91 +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.carbondata.query.merger.exception;
-
-import java.util.Locale;
-
-public class ResultMergerException extends Exception {
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public ResultMergerException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public ResultMergerException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public ResultMergerException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java b/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java
deleted file mode 100644
index e8289a9..0000000
--- a/core/src/main/java/org/carbondata/query/processor/exception/DataProcessorException.java
+++ /dev/null
@@ -1,91 +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.carbondata.query.processor.exception;
-
-import java.util.Locale;
-
-public class DataProcessorException extends Exception {
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public DataProcessorException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public DataProcessorException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public DataProcessorException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java b/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java
deleted file mode 100644
index 691a070..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/filter/CarbonFilterInfo.java
+++ /dev/null
@@ -1,182 +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.carbondata.query.queryinterface.filter;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-public class CarbonFilterInfo implements Serializable {
-  private static final long serialVersionUID = -6835223191506253050L;
-
-  protected List<String> excludedMembers;
-
-  protected List<String> includedMembers;
-
-  /**
-   * includedMembers or .
-   */
-  protected List<String> includedOrMembers;
-
-  /**
-   * CarbonFilterInfo.
-   */
-  public CarbonFilterInfo() {
-    super();
-    this.excludedMembers = new ArrayList<String>(10);
-    this.includedMembers = new ArrayList<String>(10);
-    this.includedOrMembers = new ArrayList<String>(10);
-  }
-
-  /**
-   * CarbonFilterInfo.
-   *
-   * @param exludedMembers
-   * @param includedMembers
-   */
-  public CarbonFilterInfo(List<String> exludedMembers, List<String> includedMembers) {
-    super();
-    this.excludedMembers = (null == exludedMembers ? new ArrayList<String>(10) : exludedMembers);
-    this.includedMembers = (null == includedMembers ? new ArrayList<String>(10) : includedMembers);
-    this.includedOrMembers = new ArrayList<String>(10);
-  }
-
-  /**
-   * getExcludedMembers.
-   *
-   * @return List<String>.
-   */
-  public List<String> getExcludedMembers() {
-    return excludedMembers;
-  }
-
-  /**
-   * getIncludedMembers.
-   *
-   * @return List<String>.
-   */
-  public List<String> getIncludedMembers() {
-    return includedMembers;
-  }
-
-  /**
-   * addIncludedMembers.
-   *
-   * @param aMember
-   */
-  public void addIncludedMembers(String aMember) {
-    includedMembers.add(aMember);
-  }
-
-  /**
-   * addExcludedMembers.
-   *
-   * @param aMember
-   */
-  public void addExcludedMembers(String aMember) {
-    excludedMembers.add(aMember);
-  }
-
-  /**
-   * addIncludedMembers.
-   *
-   * @param aMember
-   */
-  public void addAllIncludedMembers(List<String> members) {
-    includedMembers.addAll(members);
-  }
-
-  /**
-   * addExcludedMembers.
-   *
-   * @param aMember
-   */
-  public void addAllExcludedMembers(List<String> members) {
-    excludedMembers.addAll(members);
-  }
-
-  /**
-   * Final filter is a intersection
-   *
-   * @return List<String>.
-   */
-  public List<String> getEffectiveIncludedMembers() {
-
-    List<String> effectiveMems = new ArrayList<String>(includedMembers);
-    effectiveMems.removeAll(excludedMembers);
-    return effectiveMems;
-  }
-
-  /**
-   * getEffectiveExcludedMembers
-   *
-   * @return List<String>.
-   */
-  public List<String> getEffectiveExcludedMembers() {
-    return includedMembers.size() > 0 ? new ArrayList<String>(10) : excludedMembers;
-  }
-
-  /* (non-Javadoc)
-   * @see java.lang.Object#hashCode()
-   */
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((excludedMembers == null) ? 0 : excludedMembers.hashCode());
-    result = prime * result + ((includedMembers == null) ? 0 : includedMembers.hashCode());
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj instanceof CarbonFilterInfo) {
-
-      if (this == obj) {
-        return true;
-      }
-
-      CarbonFilterInfo info = (CarbonFilterInfo) obj;
-
-      if (excludedMembers == null) {
-        if (info.excludedMembers != null) {
-          return false;
-        }
-      } else if (!excludedMembers.equals(info.excludedMembers)) {
-        return false;
-      }
-      if (includedMembers == null) {
-        if (info.includedMembers != null) {
-          return false;
-        }
-      } else if (!includedMembers.equals(info.includedMembers)) {
-        return false;
-      }
-      return true;
-    }
-
-    return false;
-  }
-
-  /**
-   * @return the includedOrMembers
-   */
-  public List<String> getIncludedOrMembers() {
-    return includedOrMembers;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java b/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java
deleted file mode 100644
index f65e8d1..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/CarbonQuery.java
+++ /dev/null
@@ -1,233 +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.carbondata.query.queryinterface.query;
-
-import java.io.Serializable;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevel;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevelFilter;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasure;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasureFilter;
-
-public interface CarbonQuery extends Serializable {
-  /**
-   * This property can be set to the extra properties Map
-   * It takes boolean(true or false). it enables/disables pagination.
-   */
-  String PAGINATION_REQUIRED = "PAGINATION_REQUIRED";
-
-  /**
-   * This property can be set to the extra properties Map
-   * It takes string in the following format 0-100 or 1000-1100.
-   * This property can be taken only if PAGINATION_REQUIRED set to true;
-   */
-  String PAGE_RANGE = "PAGE_RANGE";
-
-  /**
-   * This property can be set to the extra properties Map
-   * It takes unique string and also this property can be taken only if PAGINATION_REQUIRED
-   * set to true;.
-   */
-  String QUERY_ID = "QUERY_ID";
-
-  /**
-   * Property can be set to save the result as a Data Set
-   */
-  String DATA_SET_PATH = "DATA_SET_PATH";
-
-  /**
-   * Property can be set to configure the transformations in the query
-   */
-  String TRANSFORMATIONS = "TRANSFORMATIONS";
-
-  /**
-   * Add dimension levels to the query
-   *
-   * @param dimensionLevel
-   * @param axis
-   */
-  void addDimensionLevel(CarbonDimensionLevel dimensionLevel, CarbonDimensionLevelFilter filter,
-      SortType sortType, AxisType axis);
-
-  /**
-   * Add measure to the query
-   *
-   * @param measure
-   * @param axis
-   */
-  void addMeasure(CarbonMeasure measure, List<CarbonMeasureFilter> filters, SortType sortType);
-
-  /**
-   * Add level filter to the query. If this dimension level is already added to any axis,then no
-   * need to add again.
-   *
-   * @param dimensionLevel
-   */
-  void addSlice(CarbonDimensionLevel dimensionLevel, CarbonDimensionLevelFilter filter);
-
-  /**
-   * Add level filter to the query. If this measure is already added to any axis,then no need
-   * to add again.
-   *
-   * @param CarbonMeasure measure
-   */
-  void addSlice(CarbonMeasure measure, List<CarbonMeasureFilter> filters);
-
-  /**
-   * Add top count to the query
-   *
-   * @param dimensionLevel
-   * @param measure
-   * @param count
-   */
-  void addTopCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure, int count);
-
-  /**
-   * Add bottom count to the query.
-   *
-   * @param dimensionLevel
-   * @param measure
-   * @param count
-   */
-  void addBottomCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure, int count);
-
-  /**
-   * Whether to show dimension properties or not.
-   *
-   * @param showProperties
-   */
-  void showLevelProperties(boolean showProperties);
-
-  /**
-   * When it set as true then user needs to provide the filters exactly with there parent members.
-   * For example : To apply day level filter as 1 then he has to provide like [2000].[jan].[1].
-   * Now it exactly fetches the data for that day
-   * If it is false then he can provide just like [1].
-   * But this will fetch the data for the day 1 of all months
-   * and years.
-   *
-   * @param exactLevelsMatch. By default it is false.
-   */
-  void setExactHirarchyLevelsMatch(boolean exactLevelsMatch);
-
-  /**
-   * This is properties will available to the execution.
-   * This is only used from Analyzer client purpose.
-   *
-   * @param extraProperties
-   */
-  void setExtraProperties(Map<String, Object> extraProperties);
-
-  /**
-   * When this property sets, it converts the data to the target time zone.
-   * By default there is no time zone set. When this property sets, even the time filters
-   * passed through this interface will be converted from this timezone.
-   * For example : timezone sets UTC-12 and filters passed are Jan 1 23:00 then it
-   * converts filter to Jan 2 11:00.
-   *
-   * @param timeZone
-   */
-  void setTimeZone(String timeZone);
-
-  /**
-   * Axis
-   */
-  public enum AxisType {
-    /**
-     * Row axis
-     */
-    ROW(0),
-    /**
-     * Column axis
-     */
-    COLUMN(1),
-
-    /**
-     * SLICE
-     */
-    SLICE(2);
-
-    /**
-     * index
-     */
-    private int index;
-
-    /**
-     * Get axis type
-     *
-     * @param index
-     */
-    private AxisType(int index) {
-      this.index = index;
-    }
-
-    /**
-     * Get axis index
-     *
-     * @return index
-     */
-    public int getIndex() {
-      return index;
-    }
-  }
-
-  /**
-   * Sort type
-   */
-  public enum SortType {
-    /**
-     * Ascending order
-     */
-    ASC(0),
-    /**
-     * Descending order
-     */
-    DESC(1),
-    /**
-     * Ascending order
-     */
-    BASC(2),
-    /**
-     * Descending order
-     */
-    BDESC(3),
-    /**
-     * None
-     */
-    NONE(-1);
-
-    private int sortVal;
-
-    SortType(int sortVal) {
-      this.sortVal = sortVal;
-    }
-
-    /**
-     * getSortValue
-     *
-     * @return
-     */
-    public int getSortValue() {
-      return sortVal;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java b/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java
deleted file mode 100644
index b12057f..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/impl/CarbonQueryImpl.java
+++ /dev/null
@@ -1,223 +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.carbondata.query.queryinterface.query.impl;
-
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.query.queryinterface.query.CarbonQuery;
-import org.carbondata.query.queryinterface.query.metadata.Axis;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevel;
-import org.carbondata.query.queryinterface.query.metadata.CarbonDimensionLevelFilter;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasure;
-import org.carbondata.query.queryinterface.query.metadata.CarbonMeasureFilter;
-import org.carbondata.query.queryinterface.query.metadata.TopCount;
-
-/**
- * It is the implementation class for CarbonQuery interface.
- */
-public class CarbonQueryImpl implements CarbonQuery {
-  private static final long serialVersionUID = -1565369538375956018L;
-
-  /**
-   * Slice number
-   */
-  private static final int SLICE = 2;
-
-  /**
-   * axises
-   */
-  private Axis[] axises;
-
-  /**
-   * Top count list
-   */
-  private List<TopCount> topCounts = new ArrayList<TopCount>(10);
-
-  /**
-   * propertiesRequired
-   */
-  private boolean propertiesRequired;
-
-  /**
-   * When it set as true then user needs to provide the filters exactly with there parent members.
-   */
-  private boolean exactLevelsMatch;
-
-  /**
-   * breakHierarchyTopN
-   */
-  private boolean breakHierarchyTopN;
-
-  /**
-   * Time zone to covert the data.
-   */
-  private String timeZone;
-
-  private Map<String, Object> extraProperties = new HashMap<String, Object>(16);
-
-  /**
-   * Default constructor
-   */
-  public CarbonQueryImpl() {
-    axises = new Axis[3];
-    for (int i = 0; i < axises.length; i++) {
-      axises[i] = new Axis();
-    }
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addDimensionLevel(CarbonDimensionLevel dimensionLevel,
-      CarbonDimensionLevelFilter filter, SortType sortType, AxisType axis) {
-    sortType = sortType == null ? SortType.NONE : sortType;
-    axises[axis.getIndex()].add(dimensionLevel, sortType, null, filter);
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addMeasure(CarbonMeasure measure, List<CarbonMeasureFilter> filters,
-      SortType sortType) {
-    sortType = sortType == null ? SortType.NONE : sortType;
-    axises[AxisType.COLUMN.getIndex()].add(measure, sortType, filters, null);
-
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addSlice(CarbonDimensionLevel dimensionLevel,
-      CarbonDimensionLevelFilter filter) {
-    axises[SLICE].add(dimensionLevel, null, null, filter);
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addSlice(CarbonMeasure measure, List<CarbonMeasureFilter> filters) {
-    axises[SLICE].add(measure, null, filters, null);
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addTopCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure,
-      int count) {
-    topCounts.add(new TopCount(dimensionLevel, measure, count, TopCount.TopNType.TOP));
-  }
-
-  /**
-   * see interface comments.
-   */
-  @Override public void addBottomCount(CarbonDimensionLevel dimensionLevel, CarbonMeasure measure,
-      int count) {
-    topCounts.add(new TopCount(dimensionLevel, measure, count, TopCount.TopNType.BOTTOM));
-  }
-
-  /**
-   * @return the axises
-   */
-  public Axis[] getAxises() {
-    return axises;
-  }
-
-  /**
-   * @return the topCounts
-   */
-  public List<TopCount> getTopCounts() {
-    return topCounts;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public void showLevelProperties(boolean showProerties) {
-    propertiesRequired = showProerties;
-  }
-
-  /**
-   * Whether can show properties or not.
-   *
-   * @return
-   */
-  public boolean isShowLevelProperties() {
-    return propertiesRequired;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public void setExactHirarchyLevelsMatch(boolean exactLevelsMatch) {
-    this.exactLevelsMatch = exactLevelsMatch;
-  }
-
-  /**
-   * @return the exactLevelsMatch
-   */
-  public boolean isExactLevelsMatch() {
-    return exactLevelsMatch;
-  }
-
-  /**
-   * @return the extraProperties
-   */
-  public Map<String, Object> getExtraProperties() {
-    return extraProperties;
-  }
-
-  @Override public void setExtraProperties(Map<String, Object> extraProperties) {
-    this.extraProperties = extraProperties;
-  }
-
-  /**
-   * @return the breakHierarchyTopN
-   */
-  public boolean isBreakHierarchyTopN() {
-    return breakHierarchyTopN;
-  }
-
-  /**
-   * @param breakHierarchyTopN the breakHierarchyTopN to set
-   */
-  public void setBreakHierarchyTopN(boolean breakHierarchyTopN) {
-    this.breakHierarchyTopN = breakHierarchyTopN;
-  }
-
-  /**
-   * getTimeZone
-   *
-   * @return
-   */
-  public String getTimeZone() {
-    return this.timeZone;
-  }
-
-  /**
-   * setTimeZone
-   */
-  @Override public void setTimeZone(String timeZone) {
-    this.timeZone = timeZone;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java
deleted file mode 100644
index 9ee9edf..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/AbstractCarbonLevel.java
+++ /dev/null
@@ -1,57 +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.carbondata.query.queryinterface.query.metadata;
-
-/**
- * It is abstract class for CarbonLevel interface.
- */
-public abstract class AbstractCarbonLevel implements CarbonLevel {
-  private static final long serialVersionUID = -1487270452433379657L;
-
-  private boolean visible = true;
-
-  /**
-   * See interface comments
-   */
-  @Override public String getDimensionName() {
-    return null;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public String getHierarchyName() {
-    return null;
-  }
-
-  /**
-   * @return the visible
-   */
-  public boolean isVisible() {
-    return visible;
-  }
-
-  /**
-   * @param visible the visible to set
-   */
-  public void setVisible(boolean visible) {
-    this.visible = visible;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java
deleted file mode 100644
index 711fab8..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/Axis.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.query.queryinterface.query.CarbonQuery;
-
-/**
- * It is Axis class, it can be row,column or slice axis.It contains all information of
- * query depends on levels and measures added in query.
- */
-public class Axis implements Serializable {
-  private static final long serialVersionUID = -574689684553603640L;
-
-  private List<CarbonLevelHolder> dims = new ArrayList<CarbonLevelHolder>(10);
-
-  /**
-   * Add query details to this axis.
-   *
-   * @param level
-   * @param sortType
-   * @param msrFilters
-   * @param dimLevelFilter
-   */
-  public void add(CarbonLevel level, CarbonQuery.SortType sortType,
-      List<CarbonMeasureFilter> msrFilters, CarbonDimensionLevelFilter dimLevelFilter) {
-    CarbonLevelHolder holder = new CarbonLevelHolder(level, sortType);
-    holder.setMsrFilters(msrFilters);
-    holder.setDimLevelFilter(dimLevelFilter);
-    dims.add(holder);
-  }
-
-  /**
-   * Get dims
-   *
-   * @return the dims
-   */
-  public List<CarbonLevelHolder> getDims() {
-    return dims;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java
deleted file mode 100644
index 7f2c0cf..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonCalculatedMeasure.java
+++ /dev/null
@@ -1,99 +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.carbondata.query.queryinterface.query.metadata;
-
-/**
- * Calculated measures can be created by using this class
- */
-public class CarbonCalculatedMeasure extends CarbonMeasure {
-  private static final long serialVersionUID = 4176313704077360543L;
-  private String expression;
-  private boolean groupCount;
-  private String groupDimensionFormula;
-  private CarbonDimensionLevel groupDimensionLevel;
-
-  public CarbonCalculatedMeasure(String measureName, String expr) {
-    super(measureName);
-  }
-
-  /**
-   * @return the expression
-   */
-  public String getExpression() {
-    return expression;
-  }
-
-  /**
-   * @param expression the expression to set
-   */
-  public void setExpression(String expression) {
-    this.expression = expression;
-  }
-
-  /**
-   * @return Returns the groupCount.
-   */
-  public boolean isGroupCount() {
-    return groupCount;
-  }
-
-  /**
-   * @param groupCount The groupCount to set.
-   */
-  public void setGroupCount(boolean groupCount) {
-    this.groupCount = groupCount;
-  }
-
-  /**
-   * @return Returns the groupDimensionFormula.
-   */
-  public String getGroupDimensionFormula() {
-    return groupDimensionFormula;
-  }
-
-  /**
-   * @param groupDimensionFormula The groupDimensionFormula to set.
-   */
-  public void setGroupDimensionFormula(String groupDimensionFormula) {
-    this.groupDimensionFormula = groupDimensionFormula;
-  }
-
-  /**
-   * @return Returns the groupDimensionLevel.
-   */
-  public CarbonDimensionLevel getGroupDimensionLevel() {
-    return groupDimensionLevel;
-  }
-
-  /**
-   * @param groupDimensionLevel The groupDimensionLevel to set.
-   */
-  public void setGroupDimensionLevel(CarbonDimensionLevel groupDimensionLevel) {
-    this.groupDimensionLevel = groupDimensionLevel;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public CarbonLevelType getType() {
-    return CarbonLevelType.CALCULATED_MEASURE;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java
deleted file mode 100644
index a2fcac6..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevel.java
+++ /dev/null
@@ -1,81 +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.carbondata.query.queryinterface.query.metadata;
-
-/**
- * Carbon dimension level;
- */
-public class CarbonDimensionLevel extends AbstractCarbonLevel {
-  private static final long serialVersionUID = 4012085091766940643L;
-
-  /**
-   * Dimension name
-   */
-  private String dimensionName;
-
-  /**
-   * Hierarchy name
-   */
-  private String hierarchyName;
-
-  /**
-   * level name
-   */
-  private String levelName;
-
-  /**
-   * Constructor
-   *
-   * @param dimensionName
-   * @param hierarchyName
-   * @param levelName
-   */
-  public CarbonDimensionLevel(String dimensionName, String hierarchyName, String levelName) {
-    this.dimensionName = dimensionName;
-    this.hierarchyName = hierarchyName;
-    this.levelName = levelName;
-  }
-
-  /**
-   * @return the dimensionName
-   */
-  public String getDimensionName() {
-    return dimensionName;
-  }
-
-  /**
-   * @return the hierarchyName
-   */
-  public String getHierarchyName() {
-    return hierarchyName;
-  }
-
-  /**
-   * @return the levelName
-   */
-  public String getName() {
-    return levelName;
-  }
-
-  @Override public CarbonLevelType getType() {
-
-    return CarbonLevelType.DIMENSION;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java
deleted file mode 100644
index 6c85b6e..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonDimensionLevelFilter.java
+++ /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.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-/**
- * Level filter
- */
-public class CarbonDimensionLevelFilter implements Serializable {
-  private static final long serialVersionUID = 5028332445998450964L;
-
-  /**
-   * Include filter
-   */
-  private List<Object> includeFilter = new ArrayList<Object>(10);
-
-  /**
-   * Exclude filter
-   */
-  private List<Object> excludeFilter = new ArrayList<Object>(10);
-
-  /**
-   * Contains filter
-   */
-  private List<String> containsFilter = new ArrayList<String>(10);
-
-  /**
-   * Does not contain filter
-   */
-  private List<String> doesNotContainsFilter = new ArrayList<String>(10);
-
-  /**
-   * afterTopN
-   */
-  private boolean afterTopN;
-
-  /**
-   * @return the includeFilter
-   */
-  public List<Object> getIncludeFilter() {
-    return includeFilter;
-  }
-
-  /**
-   * Add filters append  with braces []. Like [2000]
-   *
-   * @param includeFilter the includeFilter to set
-   */
-  public void setIncludeFilter(List<Object> includeFilter) {
-    this.includeFilter = includeFilter;
-  }
-
-  /**
-   * @return the excludeFilter
-   */
-  public List<Object> getExcludeFilter() {
-    return excludeFilter;
-  }
-
-  /**
-   * Add filters append  with braces []. Like [2000]
-   *
-   * @param excludeFilter the excludeFilter to set
-   */
-  public void setExcludeFilter(List<Object> excludeFilter) {
-    this.excludeFilter = excludeFilter;
-  }
-
-  /**
-   * @return the containsFilter
-   */
-  public List<String> getContainsFilter() {
-    return containsFilter;
-  }
-
-  /**
-   * This filter does not work along with CarbonQuery.setExactHirarchyLevelsMatch set as true.
-   *
-   * @param containsFilter the containsFilter to set
-   */
-  public void setContainsFilter(List<String> containsFilter) {
-    this.containsFilter = containsFilter;
-  }
-
-  /**
-   * @return the doesNotContainsFilter
-   */
-  public List<String> getDoesNotContainsFilter() {
-    return doesNotContainsFilter;
-  }
-
-  /**
-   * @param doesNotContainsFilter the doesNotContainsFilter to set
-   */
-  public void setDoesNotContainsFilter(List<String> doesNotContainsFilter) {
-    this.doesNotContainsFilter = doesNotContainsFilter;
-  }
-
-  /**
-   * @return the afterTopN
-   */
-  public boolean isAfterTopN() {
-    return afterTopN;
-  }
-
-  /**
-   * @param afterTopN the afterTopN to set
-   */
-  public void setAfterTopN(boolean afterTopN) {
-    this.afterTopN = afterTopN;
-  }
-
-  public String toSQLConstruct(String levelName) {
-    StringBuffer buffer = new StringBuffer();
-
-    boolean appendAndRequired = false;
-
-    // Include filters list
-    if (includeFilter.size() > 1) {
-      buffer.append(levelName);
-      buffer.append(" IN ( ");
-      for (int i = 0; i < includeFilter.size(); i++) {
-        buffer.append("'" + includeFilter.get(i) + "'");
-        if (i != includeFilter.size() - 1) {
-          buffer.append(" , ");
-        }
-      }
-      buffer.append(" ) ");
-
-      appendAndRequired = true;
-    } else if (includeFilter.size() > 0) {
-      buffer.append(levelName);
-      buffer.append(" = '" + includeFilter.get(0) + "'");
-
-      appendAndRequired = true;
-    }
-
-    // Exclude filters list
-    if (excludeFilter.size() > 1) {
-      if (appendAndRequired) {
-        buffer.append(" AND ");
-      }
-      buffer.append(levelName);
-      buffer.append(" NOT IN (");
-      for (int i = 0; i < excludeFilter.size(); i++) {
-        buffer.append("'" + excludeFilter.get(i) + "'");
-        if (i != excludeFilter.size() - 1) {
-          buffer.append(" , ");
-        }
-      }
-      buffer.append(" ) ");
-
-      appendAndRequired = true;
-    } else if (excludeFilter.size() > 0) {
-      if (appendAndRequired) {
-        buffer.append(" AND ");
-      }
-      buffer.append(levelName);
-      buffer.append(" != '" + excludeFilter.get(0) + "'");
-
-      appendAndRequired = true;
-    }
-
-    // Contains filters list
-    if (containsFilter.size() > 0) {
-
-      for (String containsString : containsFilter) {
-        if (appendAndRequired) {
-          buffer.append(" AND ");
-        }
-
-        buffer.append(levelName);
-        buffer.append(" LIKE ");
-        buffer.append("'" + containsString + "'");
-
-        appendAndRequired = true;
-      }
-
-    }
-
-    //Doesn't contain filter
-    if (doesNotContainsFilter.size() > 0) {
-
-      for (String containsString : doesNotContainsFilter) {
-        if (appendAndRequired) {
-          buffer.append(" AND ");
-        }
-        buffer.append(levelName);
-        buffer.append(" NOT LIKE ");
-        buffer.append("'" + containsString + "'");
-        appendAndRequired = true;
-      }
-
-    }
-    return buffer.toString();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.java
deleted file mode 100644
index 62b1180..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevel.java
+++ /dev/null
@@ -1,83 +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.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * It is level interface for CARBON dimension level and measure.
- */
-public interface CarbonLevel extends Serializable {
-
-  /**
-   * Type of level, either dimension level or measure.
-   *
-   * @return CarbonLevelType
-   */
-  CarbonLevelType getType();
-
-  /**
-   * Dimension name of the level it belonged to.
-   *
-   * @return the dimensionName
-   */
-  String getDimensionName();
-
-  /**
-   * Hierarchy name of the level it belonged to.
-   *
-   * @return the hierarchyName
-   */
-  String getHierarchyName();
-
-  /**
-   * Name of dimension level or measure
-   *
-   * @return the levelName
-   */
-  String getName();
-
-  /**
-   * Type of dimension level, either level or measure
-   *
-   * @author R00900208
-   */
-  public enum CarbonLevelType {
-    /**
-     * DIMENSION
-     */
-    DIMENSION,
-
-    /**
-     * MEASURE
-     */
-    MEASURE,
-
-    /**
-     * MEASURE
-     */
-    CALCULATED_MEASURE,
-
-    /**
-     * DYNAMIC LEVEL
-     */
-    DYNAMIC_DIMENSION;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java
deleted file mode 100644
index b6bc13e..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonLevelHolder.java
+++ /dev/null
@@ -1,113 +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.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.List;
-
-import org.carbondata.query.queryinterface.query.CarbonQuery;
-
-/**
- * It is holder class for a level
- */
-public class CarbonLevelHolder implements Serializable {
-  private static final long serialVersionUID = -6328136034161360231L;
-
-  /**
-   * Level
-   */
-  private CarbonLevel level;
-
-  /**
-   * sortType
-   */
-  private CarbonQuery.SortType sortType;
-
-  /**
-   * msrFilters
-   */
-  private List<CarbonMeasureFilter> msrFilters;
-
-  /**
-   * dimLevelFilter
-   */
-  private CarbonDimensionLevelFilter dimLevelFilter;
-
-  /**
-   * Constructor
-   *
-   * @param level
-   * @param sortType
-   */
-  public CarbonLevelHolder(CarbonLevel level, CarbonQuery.SortType sortType) {
-    super();
-    this.level = level;
-    this.sortType = sortType;
-  }
-
-  /**
-   * @return the level
-   */
-  public CarbonLevel getLevel() {
-    return level;
-  }
-
-  /**
-   * @return the sortType
-   */
-  public CarbonQuery.SortType getSortType() {
-    return sortType;
-  }
-
-  /**
-   * @param sortType the sortType to set
-   */
-  public void setSortType(CarbonQuery.SortType sortType) {
-    this.sortType = sortType;
-  }
-
-  /**
-   * @return the msrFilter
-   */
-  public List<CarbonMeasureFilter> getMsrFilters() {
-    return msrFilters;
-  }
-
-  /**
-   * @param msrFilter the msrFilter to set
-   */
-  public void setMsrFilters(List<CarbonMeasureFilter> msrFilters) {
-    this.msrFilters = msrFilters;
-  }
-
-  /**
-   * @return the dimLevelFilter
-   */
-  public CarbonDimensionLevelFilter getDimLevelFilter() {
-    return dimLevelFilter;
-  }
-
-  /**
-   * @param dimLevelFilter the dimLevelFilter to set
-   */
-  public void setDimLevelFilter(CarbonDimensionLevelFilter dimLevelFilter) {
-    this.dimLevelFilter = dimLevelFilter;
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java
deleted file mode 100644
index e96f8a2..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasure.java
+++ /dev/null
@@ -1,81 +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.carbondata.query.queryinterface.query.metadata;
-
-/**
- * Carbon Measure class
- */
-public class CarbonMeasure extends AbstractCarbonLevel {
-  private static final long serialVersionUID = 4257185028603048687L;
-
-  /**
-   * Measure name
-   */
-  private String measureName;
-
-  private CarbonDimensionLevel dimensionLevel;
-
-  /**
-   * Constructor
-   *
-   * @param measureName
-   */
-  public CarbonMeasure(String measureName) {
-    this.measureName = measureName;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param measureName
-   * @param aggregateName
-   */
-  public CarbonMeasure(String measureName, String aggregateName) {
-    this.measureName = measureName;
-  }
-
-  /**
-   * @return the measureName
-   */
-  public String getName() {
-    return measureName;
-  }
-
-  /**
-   * See interface comments
-   */
-  @Override public CarbonLevelType getType() {
-    return CarbonLevelType.MEASURE;
-  }
-
-  /**
-   * @return the dimensionLevel
-   */
-  public CarbonDimensionLevel getDimensionLevel() {
-    return dimensionLevel;
-  }
-
-  /**
-   * @param dimensionLevel the dimensionLevel to set
-   */
-  public void setDimensionLevel(CarbonDimensionLevel dimensionLevel) {
-    this.dimensionLevel = dimensionLevel;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java
deleted file mode 100644
index 7c89dac..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMeasureFilter.java
+++ /dev/null
@@ -1,142 +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.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * Measure filter
- */
-public class CarbonMeasureFilter implements Serializable {
-  private static final long serialVersionUID = -4253090536204072658L;
-
-  /**
-   * filterValue
-   */
-  private double filterValue;
-
-  /**
-   * filterType
-   */
-  private FilterType filterType;
-
-  /**
-   * afterTopN
-   */
-  private boolean afterTopN;
-
-  /**
-   * Constructor that takes filter information for measure filter.
-   *
-   * @param filterValue
-   * @param filterType
-   */
-  public CarbonMeasureFilter(double filterValue, FilterType filterType) {
-    this.filterValue = filterValue;
-    this.filterType = filterType;
-  }
-
-  /**
-   * Constructor that takes filter information for measure filter.
-   *
-   * @param filterType
-   */
-  public CarbonMeasureFilter(FilterType filterType) {
-    this.filterType = filterType;
-  }
-
-  /**
-   * get FilterValue
-   *
-   * @return the filterValue
-   */
-  public double getFilterValue() {
-    return filterValue;
-  }
-
-  /**
-   * FilterType
-   *
-   * @return the filterType
-   */
-  public FilterType getFilterType() {
-    return filterType;
-  }
-
-  /**
-   * @return the afterTopN
-   */
-  public boolean isAfterTopN() {
-    return afterTopN;
-  }
-
-  /**
-   * @param afterTopN the afterTopN to set
-   */
-  public void setAfterTopN(boolean afterTopN) {
-    this.afterTopN = afterTopN;
-  }
-
-  public String toSQLConstruct(String levelName) {
-    return levelName + filterType.symbol + filterValue;
-  }
-
-  /**
-   * '
-   * Enum for measure filter types.
-   */
-  public enum FilterType {
-    /**
-     * EQUAL_TO
-     */
-    EQUAL_TO(" = "),
-    /**
-     * NOT_EQUAL_TO
-     */
-    NOT_EQUAL_TO(" != "),
-    /**
-     * GREATER_THAN
-     */
-    GREATER_THAN(" > "),
-    /**
-     * LESS_THAN
-     */
-    LESS_THAN(" < "),
-    /**
-     * LESS_THAN_EQUAL
-     */
-    LESS_THAN_EQUAL(" <= "),
-    /**
-     * GREATER_THAN_EQUAL
-     */
-    GREATER_THAN_EQUAL(" >= "),
-
-    /**
-     * NOT_EMPTY
-     */
-    NOT_EMPTY(" IS NOT NULL ");
-
-    String symbol;
-
-    FilterType(String symbol) {
-      this.symbol = symbol;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java
deleted file mode 100644
index 294567c..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonMember.java
+++ /dev/null
@@ -1,92 +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.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * It is the Member object which holds information of each member which contained in query result.
- */
-public class CarbonMember implements Serializable {
-  private static final long serialVersionUID = 2149598237303284053L;
-
-  private Object name;
-
-  private Object[] properties;
-
-  /**
-   * Constructor that takes filter information for each member.
-   *
-   * @param name
-   * @param properties
-   */
-  public CarbonMember(Object name, Object[] properties) {
-    this.name = name;
-    this.properties = properties;
-  }
-
-  /**
-   * @return the name
-   */
-  public Object getName() {
-    return name;
-  }
-
-  /**
-   * @return the properties
-   */
-  public Object[] getProperties() {
-    return properties;
-  }
-
-  /**
-   * @return the properties
-   */
-  public void setProperties(Object[] props) {
-    this.properties = props;
-  }
-
-  @Override public String toString() {
-    return name != null ? name.toString() : "";
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((name == null) ? 0 : name.hashCode());
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj instanceof CarbonMember) {
-      if (this == obj) {
-        return true;
-      }
-
-      CarbonMember other = (CarbonMember) obj;
-      if (!(name == null ? other.name == null : name.equals(other.name))) {
-        return false;
-      }
-      return true;
-
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java
deleted file mode 100644
index 048b7b8..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/CarbonTuple.java
+++ /dev/null
@@ -1,76 +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.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-import java.util.Arrays;
-
-/**
- * CarbonTuple class , it contains the each row or column information of query result.
- */
-public class CarbonTuple implements Serializable {
-  private static final long serialVersionUID = 6432454407461679716L;
-
-  private CarbonMember[] tuple;
-
-  public CarbonTuple(CarbonMember[] tuple) {
-    this.tuple = tuple;
-  }
-
-  /**
-   * Size of tuple.
-   *
-   * @return
-   */
-  public int size() {
-    return tuple.length;
-  }
-
-  /**
-   * Get all members inside tuple.
-   *
-   * @return
-   */
-  public CarbonMember[] getTuple() {
-    return tuple;
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + Arrays.hashCode(tuple);
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj instanceof CarbonTuple) {
-      if (this == obj) {
-        return true;
-      }
-      CarbonTuple other = (CarbonTuple) obj;
-      if (!Arrays.equals(tuple, other.tuple)) {
-        return false;
-      }
-      return true;
-
-    }
-    return false;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.java
deleted file mode 100644
index b8f0eff..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/DSLTransformation.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.carbondata.query.queryinterface.query.metadata;
-
-/**
- * Represents DSL transformation which can be added to <code>CarbonQuery<code>
- */
-public class DSLTransformation {
-  /**
-   * Name of the transformation.
-   */
-  private String name;
-
-  /**
-   * DSL script
-   */
-  private String dslExpression;
-
-  /**
-   * The new column name if the DSL script is adding one new column
-   */
-  private String newColumnName;
-
-  /**
-   * Flag to set if the transformation script will resulting to add a new column in
-   * the original result.
-   */
-  private boolean addAsColumn;
-
-  public DSLTransformation(String name, String dslExpression, String newColumnName,
-      boolean addAsColumn) {
-    this.name = name;
-    this.dslExpression = dslExpression;
-    this.newColumnName = newColumnName;
-    this.addAsColumn = addAsColumn;
-  }
-
-  /**
-   * @return Returns the name.
-   */
-  public String getName() {
-    return name;
-  }
-
-  /**
-   * @param name The name to set.
-   */
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /**
-   * @return Returns the dslExpression.
-   */
-  public String getDslExpression() {
-    return dslExpression;
-  }
-
-  /**
-   * @param dslExpression The dslExpression to set.
-   */
-  public void setDslExpression(String dslExpression) {
-    this.dslExpression = dslExpression;
-  }
-
-  /**
-   * @return Returns the newColumnName.
-   */
-  public String getNewColumnName() {
-    return newColumnName;
-  }
-
-  /**
-   * @param newColumnName The newColumnName to set.
-   */
-  public void setNewColumnName(String newColumnName) {
-    this.newColumnName = newColumnName;
-  }
-
-  /**
-   * @return Returns the addAsColumn.
-   */
-  public boolean isAddAsColumn() {
-    return addAsColumn;
-  }
-
-  /**
-   * @param addAsColumn The addAsColumn to set.
-   */
-  public void setAddAsColumn(boolean addAsColumn) {
-    this.addAsColumn = addAsColumn;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/29360501/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java b/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java
deleted file mode 100644
index 71f83da..0000000
--- a/core/src/main/java/org/carbondata/query/queryinterface/query/metadata/TopCount.java
+++ /dev/null
@@ -1,110 +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.carbondata.query.queryinterface.query.metadata;
-
-import java.io.Serializable;
-
-/**
- * It is top count meta class
- */
-public class TopCount implements Serializable {
-  /**
-   *
-   */
-  private static final long serialVersionUID = -8571684898961076954L;
-
-  /**
-   * CarbonDimensionLevel
-   */
-  private CarbonDimensionLevel level;
-
-  /**
-   * Measure
-   */
-  private CarbonMeasure msr;
-
-  /**
-   * TopN count
-   */
-  private int count;
-
-  /**
-   * TopN type
-   */
-  private TopNType type;
-
-  public TopCount(CarbonDimensionLevel level, CarbonMeasure msr, int count, TopNType type) {
-    this.level = level;
-    this.msr = msr;
-    this.count = count;
-    this.type = type;
-  }
-
-  /**
-   * Get level
-   *
-   * @return the level
-   */
-  public CarbonDimensionLevel getLevel() {
-    return level;
-  }
-
-  /**
-   * get measure
-   *
-   * @return the msr
-   */
-  public CarbonMeasure getMsr() {
-    return msr;
-  }
-
-  /**
-   * Get top count
-   *
-   * @return the count
-   */
-  public int getCount() {
-    return count;
-  }
-
-  /**
-   * Get the topn type
-   *
-   * @return the type
-   */
-  public TopNType getType() {
-    return type;
-  }
-
-  /**
-   * Enum for TopN types
-   */
-  public enum TopNType {
-    /**
-     * Top
-     */
-    TOP,
-    /**
-     * Bottom
-     */
-    BOTTOM;
-  }
-}
-



[50/56] [abbrv] incubator-carbondata git commit: Fixing issues after merge

Posted by jb...@apache.org.
Fixing issues after merge


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

Branch: refs/heads/master
Commit: a76ca0f2a23dca68f9fe585aef2b81127ccfecbb
Parents: 9d89d69
Author: ravipesala <ra...@gmail.com>
Authored: Tue Jun 21 00:16:30 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Tue Jun 21 00:16:30 2016 +0530

----------------------------------------------------------------------
 .../src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a76ca0f2/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index 40f63c5..4209c5a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -1325,9 +1325,10 @@ class CarbonSqlParser()
     }
 
   protected lazy val segmentId: Parser[String] =
-    ( numericLit ^^ { u => u } |
-      elem("decimal", _.isInstanceOf[lexical.FloatLit]) ^^ (_.chars)
-      )
+    numericLit ^^ { u => u } |
+      elem("decimal", p => {
+        p.getClass.getSimpleName.equals("FloatLit") ||
+        p.getClass.getSimpleName.equals("DecimalLit") } ) ^^ (_.chars)
 
   protected lazy val deleteLoadsByID: Parser[LogicalPlan] =
     DELETE ~> (LOAD|SEGMENT) ~> repsep(segmentId, ",") ~ (FROM ~> (CUBE | TABLE) ~>


[04/56] [abbrv] incubator-carbondata git commit: [Issue 618]Supported Spark 1.6 in Carbondata (#670)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java
new file mode 100644
index 0000000..b901878
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumBigDecimalAggregator.java
@@ -0,0 +1,179 @@
+/*
+ * 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.carbondata.query.aggregator.impl.sum;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.math.BigDecimal;
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.util.DataTypeUtil;
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+public class SumBigDecimalAggregator extends AbstractMeasureAggregatorBasic {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 623750056131364540L;
+
+  /**
+   * aggregate value
+   */
+  private BigDecimal aggVal;
+
+  public SumBigDecimalAggregator() {
+    aggVal = new BigDecimal(0);
+    firstTime = false;
+  }
+
+  /**
+   * This method will update the aggVal it will add new value to aggVal
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    if (firstTime) {
+      aggVal = (BigDecimal) newVal;
+      firstTime = false;
+    } else {
+      aggVal = aggVal.add((BigDecimal) newVal);
+    }
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      BigDecimal value = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
+      aggVal = aggVal.add(value);
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    if (firstTime) {
+      return new byte[0];
+    }
+    byte[] bytes = DataTypeUtil.bigDecimalToByte(aggVal);
+    ByteBuffer allocate = ByteBuffer.allocate(4 + bytes.length);
+
+    allocate.putInt(bytes.length);
+    allocate.put(bytes);
+    allocate.rewind();
+    return allocate.array();
+  }
+
+  /**
+   * This method will return aggVal
+   *
+   * @return sum value
+   */
+  @Override public BigDecimal getBigDecimalValue() {
+    return aggVal;
+  }
+
+  /* Merge the value, it will update the sum aggregate value it will add new
+   * value to aggVal
+   *
+   * @param aggregator
+   *            SumAggregator
+   *
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    if (!aggregator.isFirstTime()) {
+      agg(aggregator.getBigDecimalValue());
+    }
+  }
+
+  /**
+   * This method return the sum value as an object
+   *
+   * @return sum value as an object
+   */
+  @Override public Object getValueObject() {
+    return aggVal;
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (BigDecimal) newValue;
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    firstTime = inPut.readBoolean();
+    aggVal = new BigDecimal(inPut.readUTF());
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+    output.writeBoolean(firstTime);
+    output.writeUTF(aggVal.toString());
+
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    SumBigDecimalAggregator aggr = new SumBigDecimalAggregator();
+    aggr.aggVal = aggVal;
+    aggr.firstTime = firstTime;
+    return aggr;
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+
+    ByteBuffer buffer = ByteBuffer.wrap(value);
+    byte[] valueByte = new byte[buffer.getInt()];
+    buffer.get(valueByte);
+    BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
+    aggVal = aggVal.add(valueBigDecimal);
+    firstTime = false;
+  }
+
+  public String toString() {
+    return aggVal + "";
+  }
+
+  @Override public int compareTo(MeasureAggregator o) {
+    BigDecimal value = getBigDecimalValue();
+    BigDecimal otherVal = o.getBigDecimalValue();
+    return value.compareTo(otherVal);
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof SumBigDecimalAggregator)) {
+      return false;
+    }
+    SumBigDecimalAggregator o = (SumBigDecimalAggregator) obj;
+    return getBigDecimalValue().equals(o.getBigDecimalValue());
+  }
+
+  @Override public int hashCode() {
+    return getBigDecimalValue().hashCode();
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new SumBigDecimalAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java
new file mode 100644
index 0000000..777318d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumDoubleAggregator.java
@@ -0,0 +1,179 @@
+/*
+ * 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.carbondata.query.aggregator.impl.sum;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+public class SumDoubleAggregator extends AbstractMeasureAggregatorBasic {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 623750056131364540L;
+
+  /**
+   * aggregate value
+   */
+  private double aggVal;
+
+  /**
+   * This method will update the aggVal it will add new value to aggVal
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(double newVal) {
+    aggVal += newVal;
+    firstTime = false;
+  }
+
+  /**
+   * This method will update the aggVal it will add new value to aggVal
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    aggVal += ((Number) newVal).doubleValue();
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal+= dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    if (firstTime) {
+      return new byte[0];
+    }
+    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
+    buffer.putDouble(aggVal);
+    return buffer.array();
+  }
+
+  /**
+   * This method will return aggVal
+   *
+   * @return sum value
+   */
+
+  @Override public Double getDoubleValue() {
+    return aggVal;
+  }
+
+  /* Merge the value, it will update the sum aggregate value it will add new
+   * value to aggVal
+   *
+   * @param aggregator  SumAggregator
+   *
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    if (!aggregator.isFirstTime()) {
+      agg(aggregator.getDoubleValue());
+    }
+  }
+
+  /**
+   * This method return the sum value as an object
+   *
+   * @return sum value as an object
+   */
+  @Override public Object getValueObject() {
+    return aggVal;
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (Double) newValue;
+  }
+
+  @Override public boolean isFirstTime() {
+    return firstTime;
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    firstTime = inPut.readBoolean();
+    aggVal = inPut.readDouble();
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+    output.writeBoolean(firstTime);
+    output.writeDouble(aggVal);
+
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    SumDoubleAggregator aggr = new SumDoubleAggregator();
+    aggr.aggVal = aggVal;
+    aggr.firstTime = firstTime;
+    return aggr;
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    aggVal += ByteBuffer.wrap(value).getDouble();
+    firstTime = false;
+  }
+
+  public String toString() {
+    return aggVal + "";
+  }
+
+  @Override public int compareTo(MeasureAggregator o) {
+    double value = getDoubleValue();
+    double otherVal = o.getDoubleValue();
+    if (value > otherVal) {
+      return 1;
+    }
+    if (value < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof SumDoubleAggregator)) {
+      return false;
+    }
+    SumDoubleAggregator o = (SumDoubleAggregator) obj;
+    return getDoubleValue().equals(o.getDoubleValue());
+  }
+
+  @Override public int hashCode() {
+    return getDoubleValue().hashCode();
+  }
+
+  @Override public MeasureAggregator getNew() {
+    return new SumDoubleAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java
new file mode 100644
index 0000000..7c245d9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/sum/SumLongAggregator.java
@@ -0,0 +1,165 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.query.aggregator.impl.sum;
+
+import java.io.DataInput;
+import java.io.DataOutput;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
+
+public class SumLongAggregator extends AbstractMeasureAggregatorBasic {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 623750056131364540L;
+
+  /**
+   * aggregate value
+   */
+  private long aggVal;
+
+  /**
+   * This method will update the aggVal it will add new value to aggVal
+   *
+   * @param newVal new value
+   */
+  @Override public void agg(Object newVal) {
+    aggVal += (long) newVal;
+    firstTime = false;
+  }
+
+  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
+    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
+      aggVal+= dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
+      firstTime = false;
+    }
+  }
+
+  /**
+   * Below method will be used to get the value byte array
+   */
+  @Override public byte[] getByteArray() {
+    if (firstTime) {
+      return new byte[0];
+    }
+    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.LONG_SIZE_IN_BYTE);
+    buffer.putLong(aggVal);
+    return buffer.array();
+  }
+
+  /**
+   * This method will return aggVal
+   *
+   * @return sum value
+   */
+  @Override public Long getLongValue() {
+    return aggVal;
+  }
+
+  /* Merge the value, it will update the sum aggregate value it will add new
+   * value to aggVal
+   *
+   * @param aggregator SumAggregator
+   *
+   */
+  @Override public void merge(MeasureAggregator aggregator) {
+    if (!aggregator.isFirstTime()) {
+      agg(aggregator.getLongValue());
+    }
+  }
+
+  /**
+   * This method return the sum value as an object
+   *
+   * @return sum long value as an object
+   */
+  @Override public Object getValueObject() {
+    return aggVal;
+  }
+
+  @Override public void setNewValue(Object newValue) {
+    aggVal = (long) newValue;
+  }
+
+  @Override public void readData(DataInput inPut) throws IOException {
+    firstTime = inPut.readBoolean();
+    aggVal = inPut.readLong();
+  }
+
+  @Override public void writeData(DataOutput output) throws IOException {
+    output.writeBoolean(firstTime);
+    output.writeLong(aggVal);
+
+  }
+
+  @Override public MeasureAggregator getCopy() {
+    SumLongAggregator aggr = new SumLongAggregator();
+    aggr.aggVal = aggVal;
+    aggr.firstTime = firstTime;
+    return aggr;
+  }
+
+  @Override public void merge(byte[] value) {
+    if (0 == value.length) {
+      return;
+    }
+    aggVal += ByteBuffer.wrap(value).getLong();
+    firstTime = false;
+  }
+
+  public String toString() {
+    return aggVal + "";
+  }
+
+  @Override public int compareTo(MeasureAggregator o) {
+    Long value = getLongValue();
+    Long otherVal = o.getLongValue();
+    if (value > otherVal) {
+      return 1;
+    }
+    if (value < otherVal) {
+      return -1;
+    }
+    return 0;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if(!(obj instanceof SumLongAggregator)) {
+      return false;
+    }
+    SumLongAggregator o = (SumLongAggregator) obj;
+    return getLongValue().equals(o.getLongValue());
+  }
+
+  @Override public int hashCode() {
+    return getLongValue().hashCode();
+  }
+
+  @Override public MeasureAggregator getNew() {
+    // TODO Auto-generated method stub
+    return new SumLongAggregator();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java b/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
index 483392e..b46c4de 100644
--- a/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
+++ b/core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
@@ -24,28 +24,28 @@ import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.query.aggregator.CustomMeasureAggregator;
 import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.AvgBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.AvgDoubleAggregator;
-import org.carbondata.query.aggregator.impl.AvgLongAggregator;
-import org.carbondata.query.aggregator.impl.CountAggregator;
-import org.carbondata.query.aggregator.impl.DistinctCountAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.DistinctCountBigDecimalAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.DistinctCountLongAggregatorObjectSet;
-import org.carbondata.query.aggregator.impl.DummyBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.DummyDoubleAggregator;
-import org.carbondata.query.aggregator.impl.DummyLongAggregator;
-import org.carbondata.query.aggregator.impl.MaxAggregator;
-import org.carbondata.query.aggregator.impl.MaxBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.MaxLongAggregator;
-import org.carbondata.query.aggregator.impl.MinAggregator;
-import org.carbondata.query.aggregator.impl.MinBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.MinLongAggregator;
-import org.carbondata.query.aggregator.impl.SumBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.SumDistinctBigDecimalAggregator;
-import org.carbondata.query.aggregator.impl.SumDistinctDoubleAggregator;
-import org.carbondata.query.aggregator.impl.SumDistinctLongAggregator;
-import org.carbondata.query.aggregator.impl.SumDoubleAggregator;
-import org.carbondata.query.aggregator.impl.SumLongAggregator;
+import org.carbondata.query.aggregator.impl.avg.AvgBigDecimalAggregator;
+import org.carbondata.query.aggregator.impl.avg.AvgDoubleAggregator;
+import org.carbondata.query.aggregator.impl.avg.AvgLongAggregator;
+import org.carbondata.query.aggregator.impl.count.CountAggregator;
+import org.carbondata.query.aggregator.impl.distinct.DistinctCountAggregatorObjectSet;
+import org.carbondata.query.aggregator.impl.distinct.DistinctCountBigDecimalAggregatorObjectSet;
+import org.carbondata.query.aggregator.impl.distinct.DistinctCountLongAggregatorObjectSet;
+import org.carbondata.query.aggregator.impl.distinct.SumDistinctBigDecimalAggregator;
+import org.carbondata.query.aggregator.impl.distinct.SumDistinctDoubleAggregator;
+import org.carbondata.query.aggregator.impl.distinct.SumDistinctLongAggregator;
+import org.carbondata.query.aggregator.impl.dummy.DummyBigDecimalAggregator;
+import org.carbondata.query.aggregator.impl.dummy.DummyDoubleAggregator;
+import org.carbondata.query.aggregator.impl.dummy.DummyLongAggregator;
+import org.carbondata.query.aggregator.impl.max.MaxAggregator;
+import org.carbondata.query.aggregator.impl.max.MaxBigDecimalAggregator;
+import org.carbondata.query.aggregator.impl.max.MaxLongAggregator;
+import org.carbondata.query.aggregator.impl.min.MinAggregator;
+import org.carbondata.query.aggregator.impl.min.MinBigDecimalAggregator;
+import org.carbondata.query.aggregator.impl.min.MinLongAggregator;
+import org.carbondata.query.aggregator.impl.sum.SumBigDecimalAggregator;
+import org.carbondata.query.aggregator.impl.sum.SumDoubleAggregator;
+import org.carbondata.query.aggregator.impl.sum.SumLongAggregator;
 import org.carbondata.query.carbon.model.CustomAggregateExpression;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
new file mode 100644
index 0000000..971e4cc
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/carbon/aggregator/dimension/impl/DirectDictionaryDimensionAggregator.java
@@ -0,0 +1,114 @@
+/*
+ * 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.carbondata.query.carbon.aggregator.dimension.impl;
+
+import java.nio.ByteBuffer;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
+import org.carbondata.query.carbon.executor.util.QueryUtil;
+import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
+import org.carbondata.query.carbon.result.AbstractScannedResult;
+
+/**
+ * Class which will be used to aggregate the direct dictionary dimension data
+ */
+public class DirectDictionaryDimensionAggregator implements DimensionDataAggregator {
+
+  /**
+   * info object which store information about dimension to be aggregated
+   */
+  private DimensionAggregatorInfo dimensionAggeragtorInfo;
+
+  /**
+   * start index of the aggregator for current dimension column
+   */
+  private int aggregatorStartIndex;
+
+  /**
+   * buffer used to convert mdkey to surrogate key
+   */
+  private ByteBuffer buffer;
+
+  /**
+   * data index in the file
+   */
+  private int blockIndex;
+
+  /**
+   * to store index which will be used to aggregate
+   * number type value like sum avg
+   */
+  private int[] numberTypeAggregatorIndex;
+
+  /**
+   * DirectDictionaryGenerator
+   */
+  private DirectDictionaryGenerator directDictionaryGenerator;
+
+  /**
+   * to store index which will be used to aggregate
+   * actual type value like max, min, dictinct count
+   */
+  private int[] actualTypeAggregatorIndex;
+
+  public DirectDictionaryDimensionAggregator(DimensionAggregatorInfo dimensionAggeragtorInfo,
+      int aggregatorStartIndex, int blockIndex) {
+    this.dimensionAggeragtorInfo = dimensionAggeragtorInfo;
+    this.aggregatorStartIndex = aggregatorStartIndex;
+    this.blockIndex = blockIndex;
+    buffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
+    numberTypeAggregatorIndex =
+        QueryUtil.getNumberTypeIndex(this.dimensionAggeragtorInfo.getAggList());
+    actualTypeAggregatorIndex =
+        QueryUtil.getActualTypeIndex(this.dimensionAggeragtorInfo.getAggList());
+    directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+        .getDirectDictionaryGenerator(this.dimensionAggeragtorInfo.getDim().getDataType());
+  }
+
+  /**
+   * Below method will be used to aggregate the dimension data
+   *
+   * @param scannedResult scanned result
+   * @param aggeragtor    aggregator used to aggregate the data
+   */
+  @Override public void aggregateDimensionData(AbstractScannedResult scannedResult,
+      MeasureAggregator[] aggeragtor) {
+    byte[] dimensionData = scannedResult.getDimensionKey(blockIndex);
+    int surrogateKey = CarbonUtil.getSurrogateKey(dimensionData, buffer);
+    Object dataBasedOnDataType =
+        (long) directDictionaryGenerator.getValueFromSurrogate(surrogateKey) / 1000;
+
+    if (actualTypeAggregatorIndex.length > 0) {
+      for (int j = 0; j < actualTypeAggregatorIndex.length; j++) {
+        aggeragtor[aggregatorStartIndex + actualTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
+      }
+    }
+    if (numberTypeAggregatorIndex.length > 0) {
+      for (int j = 0; j < numberTypeAggregatorIndex.length; j++) {
+        aggeragtor[aggregatorStartIndex + numberTypeAggregatorIndex[j]].agg(dataBasedOnDataType);
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
index e13aee2..14c336d 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.carbondata.core.carbon.datastore.block.AbstractIndex;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.CountAggregator;
+import org.carbondata.query.aggregator.impl.count.CountAggregator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
index ebd90f9..789f77e 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
@@ -53,6 +53,7 @@ import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
 import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
 import org.carbondata.query.carbon.aggregator.dimension.impl.ColumnGroupDimensionsAggregator;
+import org.carbondata.query.carbon.aggregator.dimension.impl.DirectDictionaryDimensionAggregator;
 import org.carbondata.query.carbon.aggregator.dimension.impl.FixedLengthDimensionAggregator;
 import org.carbondata.query.carbon.aggregator.dimension.impl.VariableLengthDimensionAggregator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
@@ -724,9 +725,15 @@ public class QueryUtil {
         aggregatorStartIndex += numberOfAggregatorForColumnGroup;
         continue;
       } else {
+        if(CarbonUtil.hasEncoding(dim.getEncoder(), Encoding.DIRECT_DICTIONARY)){
+          dimensionDataAggregators.add(
+              new DirectDictionaryDimensionAggregator(entry.getValue().get(0),
+                  aggregatorStartIndex,
+                  dimensionToBlockIndexMapping.get(dim.getOrdinal())));
+        }
         // if it is a dictionary column than create a fixed length
         // aggeragtor
-        if (CarbonUtil
+        else if (CarbonUtil
             .hasEncoding(dim.getEncoder(), Encoding.DICTIONARY)) {
           dimensionDataAggregators.add(
               new FixedLengthDimensionAggregator(entry.getValue().get(0), null,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java
index 0957e7a..5604ecd 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/QueryResultPreparatorImpl.java
@@ -32,9 +32,9 @@ import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerat
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.CountAggregator;
-import org.carbondata.query.aggregator.impl.DistinctCountAggregator;
-import org.carbondata.query.aggregator.impl.DistinctStringCountAggregator;
+import org.carbondata.query.aggregator.impl.count.CountAggregator;
+import org.carbondata.query.aggregator.impl.distinct.DistinctCountAggregator;
+import org.carbondata.query.aggregator.impl.distinct.DistinctStringCountAggregator;
 import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
 import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
 import org.carbondata.query.carbon.model.QueryDimension;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
index 69e1d2f..a549409 100644
--- a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
@@ -63,7 +63,7 @@ object GenerateDictionaryExample {
     val tableName = carbonTableIdentifier.getTableName
     val carbonRelation = CarbonEnv.getInstance(carbonContext).carbonCatalog.
       lookupRelation1(Option(dataBaseName),
-        tableName, None) (carbonContext).asInstanceOf[CarbonRelation]
+        tableName) (carbonContext).asInstanceOf[CarbonRelation]
     val carbonTable = carbonRelation.cubeMeta.carbonTable
     val dimensions = carbonTable.getDimensionByTableName(tableName)
       .toArray.map(_.asInstanceOf[CarbonDimension])

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
----------------------------------------------------------------------
diff --git a/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala b/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
index 1cd4be4..c4f09cc 100644
--- a/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
+++ b/integration-testcases/src/test/scala/org/apache/spark/sql/common/util/QueryTest.scala
@@ -19,12 +19,13 @@ package org.apache.spark.sql.common.util
 
 import java.util.{Locale, TimeZone}
 
-import org.apache.spark.sql.{DataFrame, Row, SQLContext}
+import scala.collection.JavaConversions._
+
 import org.apache.spark.sql.catalyst.plans._
 import org.apache.spark.sql.catalyst.util._
-import org.apache.spark.sql.columnar.InMemoryRelation
+import org.apache.spark.sql.execution.columnar.InMemoryRelation
+import org.apache.spark.sql.{DataFrame, Row, SQLContext}
 
-import scala.collection.JavaConversions._
 
 class QueryTest extends PlanTest {
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala
----------------------------------------------------------------------
diff --git a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala
index ec23b80..bd7b596 100644
--- a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala
+++ b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala
@@ -81,6 +81,34 @@ class AllDataTypesTestCase1 extends QueryTest with BeforeAndAfterAll {
         "Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber," +
         "Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId," +
         "gamePointId,gamePointDescription')")
+
+      sql(
+        "create table if not exists Carbon_automation_hive (imei string,deviceInformationId int," +
+        "MAC string,deviceColor string,device_backColor string,modelId string,marketName " +
+        "string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string," +
+        "productionDate timestamp,bomCode string,internalModels string, deliveryTime string, " +
+        "channelsId string, channelsName string , deliveryAreaId string, deliveryCountry " +
+        "string, deliveryProvince string, deliveryCity string,deliveryDistrict string, " +
+        "deliveryStreet string, oxSingleNumber string,contractNumber int, ActiveCheckTime string, ActiveAreaId " +
+        "string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict" +
+        " string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, " +
+        "Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, " +
+        "Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string," +
+        "Active_webTypeDataVerNumber string, Active_operatorsVersion string, " +
+        "Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, " +
+        "Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, " +
+        "Latest_province string, Latest_city string, Latest_district string, Latest_street " +
+        "string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion " +
+        "string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion " +
+        "string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, " +
+        "Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, " +
+        "Latest_operatorId string, , gamePointId int, gamePointDescription string" +
+        ") row format delimited fields terminated by ','"
+      )
+
+      sql("LOAD DATA LOCAL INPATH '" + currentDirectory + "/src/test/resources/100_olap.csv' INTO " +
+          "table Carbon_automation_hive ")
+
     } catch {
       case e: Exception => print("ERROR: DROP Carbon_automation_test ")
     }
@@ -88,6 +116,7 @@ class AllDataTypesTestCase1 extends QueryTest with BeforeAndAfterAll {
 
   override def afterAll {
     sql("drop cube Carbon_automation_test")
+    sql("drop table Carbon_automation_hive")
 
   }
 
@@ -853,7 +882,7 @@ class AllDataTypesTestCase1 extends QueryTest with BeforeAndAfterAll {
   test("select variance(deviceInformationId) as a   from Carbon_automation_test")({
     checkAnswer(
       sql("select variance(deviceInformationId) as a   from Carbon_automation_test"),
-      Seq(Row(9.31041555963636E9))
+      sql("select variance(deviceInformationId) as a   from Carbon_automation_hive")
     )
   }
   )

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
----------------------------------------------------------------------
diff --git a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
index ab9121a..88ba722 100644
--- a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
+++ b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
@@ -81,6 +81,32 @@ class AllDataTypesTestCase2 extends QueryTest with BeforeAndAfterAll {
         "Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber," +
         "Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId," +
         "gamePointId,gamePointDescription')")
+
+      sql(
+        "create table if not exists Carbon_automation_hive2(imei string,deviceInformationId int," +
+        "MAC string,deviceColor string,device_backColor string,modelId string,marketName " +
+        "string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string," +
+        "productionDate timestamp,bomCode string,internalModels string, deliveryTime string, " +
+        "channelsId string, channelsName string , deliveryAreaId string, deliveryCountry " +
+        "string, deliveryProvince string, deliveryCity string,deliveryDistrict string, " +
+        "deliveryStreet string, oxSingleNumber string,contractNumber int, ActiveCheckTime string, ActiveAreaId " +
+        "string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict" +
+        " string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, " +
+        "Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, " +
+        "Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string," +
+        "Active_webTypeDataVerNumber string, Active_operatorsVersion string, " +
+        "Active_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, " +
+        "Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, " +
+        "Latest_province string, Latest_city string, Latest_district string, Latest_street " +
+        "string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion " +
+        "string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion " +
+        "string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, " +
+        "Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, " +
+        "Latest_operatorId string, gamePointId int,gamePointDescription string" +
+        ") row format delimited fields terminated by ','"
+      )
+      sql("LOAD DATA LOCAL INPATH '" + currentDirectory + "/src/test/resources/100_olap.csv' INTO " +
+          "table Carbon_automation_hive2 ")
     } catch {
       case e: Exception => print("ERROR: DROP Carbon_automation_test2 ")
     }
@@ -89,6 +115,7 @@ class AllDataTypesTestCase2 extends QueryTest with BeforeAndAfterAll {
   override def afterAll {
     try {
       sql("drop cube Carbon_automation_test2")
+      sql("drop table Carbon_automation_hive2")
     } catch {
       case e: Exception => print("ERROR: DROP Carbon_automation_test2 ")
     }
@@ -7902,7 +7929,9 @@ class AllDataTypesTestCase2 extends QueryTest with BeforeAndAfterAll {
       sql(
         "select variance(deviceInformationId), var_pop(imei)  from Carbon_automation_test2 where activeareaid>3"
       ),
-      Seq(Row(1.477644655616972E10, null))
+      sql(
+        "select variance(deviceInformationId), var_pop(imei)  from Carbon_automation_hive2 where activeareaid>3"
+      )
     )
   }
   )
@@ -7915,7 +7944,9 @@ class AllDataTypesTestCase2 extends QueryTest with BeforeAndAfterAll {
       sql(
         "select variance(contractNumber), var_pop(contractNumber)  from Carbon_automation_test2 where deliveryareaid>5"
       ),
-      Seq(Row(8.508651970169495E12, 8.508651970169495E12))
+      sql(
+        "select variance(contractNumber), var_pop(contractNumber)  from Carbon_automation_hive2 where deliveryareaid>5"
+      )
     )
   }
   )
@@ -7928,7 +7959,9 @@ class AllDataTypesTestCase2 extends QueryTest with BeforeAndAfterAll {
       sql(
         "select variance(AMSize), var_pop(channelsid)  from Carbon_automation_test2 where channelsid>2"
       ),
-      Seq(Row(null, 2.148423005565863))
+      sql(
+        "select variance(AMSize), var_pop(channelsid)  from Carbon_automation_hive2 where channelsid>2"
+      )
     )
   }
   )
@@ -7941,7 +7974,9 @@ class AllDataTypesTestCase2 extends QueryTest with BeforeAndAfterAll {
       sql(
         "select variance(deviceInformationId), var_pop(deviceInformationId)  from Carbon_automation_test2 where activeareaid>3"
       ),
-      Seq(Row(1.477644655616972E10, 1.477644655616972E10))
+      sql(
+        "select variance(deviceInformationId), var_pop(deviceInformationId)  from Carbon_automation_hive2 where activeareaid>3"
+      )
     )
   }
   )

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala
----------------------------------------------------------------------
diff --git a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala
index 1884e5f..12f55b7 100644
--- a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala
+++ b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala
@@ -53,7 +53,7 @@ class AllDataTypesTestCase6 extends QueryTest with BeforeAndAfterAll {
           "bomCode string,internalModels string, deliveryTime string, channelsId string, " +
           "channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince " +
           "string, deliveryCity string,deliveryDistrict string, deliveryStreet string, " +
-          "oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry " +
+          "oxSingleNumber string,contractNumber int, ActiveCheckTime string, ActiveAreaId string, ActiveCountry " +
           "string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet " +
           "string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, " +
           "Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, " +
@@ -65,8 +65,8 @@ class AllDataTypesTestCase6 extends QueryTest with BeforeAndAfterAll {
           "Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, " +
           "Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string," +
           " Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, " +
-          "Latest_phonePADPartitionedVersions string, Latest_operatorId string, " +
-          "gamePointDescription string, gamePointId int,contractNumber int) row format " +
+          "Latest_phonePADPartitionedVersions string, Latest_operatorId string,gamePointId int," +
+          "gamePointDescription string) row format " +
           "delimited fields terminated by ','"
       )
 
@@ -211,7 +211,7 @@ class AllDataTypesTestCase6 extends QueryTest with BeforeAndAfterAll {
   test("select sum( DISTINCT channelsId) a  from Carbon_automation_test6")({
     checkAnswer(
       sql("select sum( DISTINCT channelsId) a  from Carbon_automation_test6"),
-      Seq(Row(428.0)))
+      sql("select sum( DISTINCT channelsId) a  from hivetable"))
   })
 
   //TC_083
@@ -263,7 +263,7 @@ class AllDataTypesTestCase6 extends QueryTest with BeforeAndAfterAll {
   test("select variance(gamePointId) as a   from Carbon_automation_test6")({
     checkAnswer(
       sql("select variance(gamePointId) as a   from Carbon_automation_test6"),
-      Seq(Row(654787.843930927)))
+      sql("select variance(gamePointId) as a   from hivetable"))
   })
 
   //TC_120
@@ -732,14 +732,14 @@ class AllDataTypesTestCase6 extends QueryTest with BeforeAndAfterAll {
   test("select variance(gamepointid), var_pop(gamepointid)  from Carbon_automation_test6 where channelsid>2")({
     checkAnswer(
       sql("select variance(gamepointid), var_pop(gamepointid)  from Carbon_automation_test6 where channelsid>2"),
-      Seq(Row(622630.4599570761, 622630.4599570761)))
+      sql("select variance(gamepointid), var_pop(gamepointid)  from hivetable where channelsid>2"))
   })
 
   //TC_445
   test("select variance(bomcode), var_pop(gamepointid)  from Carbon_automation_test6 where activeareaid>3")({
     checkAnswer(
       sql("select variance(bomcode), var_pop(gamepointid)  from Carbon_automation_test6 where activeareaid>3"),
-      Seq(Row(1.4776446556169722E10, 663683.3954750763)))
+      sql("select variance(bomcode), var_pop(gamepointid)  from hivetable where activeareaid>3"))
   })
 
   //TC_447

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/pom.xml
----------------------------------------------------------------------
diff --git a/integration/spark/pom.xml b/integration/spark/pom.xml
index 0682a42..c98c9fb 100644
--- a/integration/spark/pom.xml
+++ b/integration/spark/pom.xml
@@ -65,17 +65,6 @@
       <version>${project.version}</version>
     </dependency>
     <dependency>
-      <groupId>eigenbase</groupId>
-      <artifactId>eigenbase-xom</artifactId>
-      <version>1.3.4</version>
-      <exclusions>
-        <exclusion>
-          <groupId>*</groupId>
-          <artifactId>*</artifactId>
-        </exclusion>
-      </exclusions>
-    </dependency>
-    <dependency>
       <groupId>it.unimi.dsi</groupId>
       <artifactId>fastutil</artifactId>
       <version>6.5.0</version>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonAggregate.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonAggregate.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonAggregate.scala
deleted file mode 100644
index 93cf675..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonAggregate.scala
+++ /dev/null
@@ -1,209 +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
-
-import java.util.HashMap
-
-import scala.Array.{canBuildFrom, fallbackCanBuildFrom}
-
-import org.apache.spark.annotation.DeveloperApi
-import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.catalyst.errors.attachTree
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.plans.physical._
-import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.execution.{SparkPlan, UnaryNode}
-
-/**
- * :: DeveloperApi ::
- * Groups input data by `groupingExpressions` and computes the `aggregateExpressions` for each
- * group.
- *
- * @param partial              if true then aggregation is done partially on local data without
- *                             shuffling to
- *                             ensure all values where `groupingExpressions` are equal are present.
- * @param groupingExpressions  expressions that are evaluated to determine grouping.
- * @param aggregateExpressions expressions that are computed for each group.
- * @param child                the input data source.
- */
-@DeveloperApi
-case class CarbonAggregate(
-    partial: Boolean,
-    groupingExpressions: Seq[Expression],
-    aggregateExpressions: Seq[NamedExpression],
-    child: SparkPlan)(@transient sqlContext: SQLContext)
-  extends UnaryNode {
-
-  override def requiredChildDistribution: Seq[Distribution] = {
-    if (partial) {
-      UnspecifiedDistribution :: Nil
-    } else {
-      if (groupingExpressions == Nil) {
-        AllTuples :: Nil
-      } else {
-        ClusteredDistribution(groupingExpressions) :: Nil
-      }
-    }
-  }
-
-  override def otherCopyArgs: Seq[AnyRef] = sqlContext :: Nil
-
-  // HACK: Generators don't correctly preserve their output through serializations so we grab
-  // out child's output attributes statically here.
-  private[this] val childOutput = child.output
-
-  override def output: Seq[Attribute] = aggregateExpressions.map(_.toAttribute)
-
-  /**
-   * An aggregate that needs to be computed for each row in a group.
-   *
-   * @param unbound         Unbound version of this aggregate, used for result substitution.
-   * @param aggregate       A bound copy of this aggregate used to create a new aggregation buffer.
-   * @param resultAttribute An attribute used to refer to the result of this aggregate in the final
-   *                        output.
-   */
-  case class ComputedAggregate(unbound: AggregateExpression1,
-      aggregate: AggregateExpression1,
-      resultAttribute: AttributeReference)
-
-  /** A list of aggregates that need to be computed for each group. */
-  private[this] val computedAggregates = aggregateExpressions.flatMap { agg =>
-    agg.collect {
-      case a: AggregateExpression1 =>
-        ComputedAggregate(
-          a,
-          BindReferences.bindReference(a, childOutput),
-          AttributeReference(s"aggResult:$a", a.dataType, a.nullable)())
-    }
-  }.toArray
-
-  /** The schema of the result of all aggregate evaluations */
-  private[this] val computedSchema = computedAggregates.map(_.resultAttribute)
-
-  /** Creates a new aggregate buffer for a group. */
-  private[this] def newAggregateBuffer(): Array[AggregateFunction1] = {
-    val buffer = new Array[AggregateFunction1](computedAggregates.length)
-    var i = 0
-    while (i < computedAggregates.length) {
-      buffer(i) = computedAggregates(i).aggregate.newInstance()
-      i += 1
-    }
-    buffer
-  }
-
-  /** Named attributes used to substitute grouping attributes into the final result. */
-  private[this] val namedGroups = groupingExpressions.map {
-    case ne: NamedExpression => ne -> ne.toAttribute
-    case e => e -> Alias(e, s"groupingExpr:$e")().toAttribute
-  }
-
-  /**
-   * A map of substitutions that are used to insert the aggregate expressions and grouping
-   * expression into the final result expression.
-   */
-  private[this] val resultMap =
-    (computedAggregates.map { agg => agg.unbound -> agg.resultAttribute } ++ namedGroups).toMap
-
-  /**
-   * Substituted version of aggregateExpressions expressions which are used to compute final
-   * output rows given a group and the result of all aggregate computations.
-   */
-  private[this] val resultExpressions = aggregateExpressions.map { agg =>
-    agg.transform {
-      case e: Expression if resultMap.contains(e) => resultMap(e)
-    }
-  }
-
-  override def doExecute(): RDD[InternalRow] = {
-    attachTree(this, "execute") {
-      if (groupingExpressions.isEmpty) {
-        child.execute().mapPartitions { iter =>
-          val buffer = newAggregateBuffer()
-          var currentRow: InternalRow = null
-          while (iter.hasNext) {
-            currentRow = iter.next()
-            var i = 0
-            while (i < buffer.length) {
-              buffer(i).update(currentRow)
-              i += 1
-            }
-          }
-          val resultProjection = new InterpretedProjection(resultExpressions, computedSchema)
-          val aggregateResults = new GenericMutableRow(computedAggregates.length)
-
-          var i = 0
-          while (i < buffer.length) {
-            aggregateResults(i) = buffer(i).eval(EmptyRow)
-            i += 1
-          }
-
-          Iterator(resultProjection(aggregateResults))
-        }
-      } else {
-        child.execute().mapPartitions { iter =>
-          val hashTable = new HashMap[InternalRow, Array[AggregateFunction1]]
-          val groupingProjection = new InterpretedMutableProjection(groupingExpressions,
-            childOutput)
-
-          var currentRow: InternalRow = null
-          while (iter.hasNext) {
-            currentRow = iter.next()
-            val currentGroup = groupingProjection(currentRow)
-            var currentBuffer = hashTable.get(currentGroup)
-            if (currentBuffer == null) {
-              currentBuffer = newAggregateBuffer()
-              hashTable.put(currentGroup.copy(), currentBuffer)
-            }
-
-            var i = 0
-            while (i < currentBuffer.length) {
-              currentBuffer(i).update(currentRow)
-              i += 1
-            }
-          }
-
-          new Iterator[InternalRow] {
-            private[this] val hashTableIter = hashTable.entrySet().iterator()
-            private[this] val aggregateResults = new GenericMutableRow(computedAggregates.length)
-            private[this] val resultProjection =
-              new InterpretedMutableProjection(resultExpressions,
-                computedSchema ++ namedGroups.map(_._2))
-            private[this] val joinedRow = new JoinedRow
-
-            override final def hasNext: Boolean = hashTableIter.hasNext
-
-            override final def next(): InternalRow = {
-              val currentEntry = hashTableIter.next()
-              val currentGroup = currentEntry.getKey
-              val currentBuffer = currentEntry.getValue
-
-              var i = 0
-              while (i < currentBuffer.length) {
-                // Evaluating an aggregate buffer returns the result.  No row is required since we
-                // already added all rows in the group using update.
-                aggregateResults(i) = currentBuffer(i).eval(EmptyRow)
-                i += 1
-              }
-              resultProjection(joinedRow(aggregateResults, currentGroup))
-            }
-          }
-        }
-      }
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index e574348..f728a32 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -19,18 +19,18 @@ package org.apache.spark.sql
 
 import scala.collection.mutable.MutableList
 
-import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.agg.{CarbonAverage, CarbonCount}
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
 import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 import org.apache.spark.sql.catalyst.expressions._
+import org.apache.spark.sql.catalyst.expressions.aggregate._
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.catalyst.plans.logical.{UnaryNode, _}
-import org.apache.spark.sql.catalyst.trees.TreeNodeRef
 import org.apache.spark.sql.execution.command.tableModel
+import org.apache.spark.sql.execution.datasources.LogicalRelation
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
-import org.apache.spark.sql.types.{BooleanType, DataType, StringType, TimestampType}
-
-import org.carbondata.spark.agg._
+import org.apache.spark.sql.types._
 
 /**
  * Top command
@@ -94,7 +94,7 @@ case class ShowCubeCommand(schemaNameOp: Option[String]) extends LogicalPlan wit
   override def children: Seq[LogicalPlan] = Seq.empty
 
   override def output: Seq[Attribute] = {
-    Seq(AttributeReference("cubeName", StringType, nullable = false)(),
+    Seq(AttributeReference("tableName", StringType, nullable = false)(),
       AttributeReference("isRegisteredWithSpark", BooleanType, nullable = false)())
   }
 }
@@ -107,8 +107,8 @@ case class ShowAllCubeCommand() extends LogicalPlan with Command {
   override def children: Seq[LogicalPlan] = Seq.empty
 
   override def output: Seq[Attribute] = {
-    Seq(AttributeReference("schemaName", StringType, nullable = false)(),
-      AttributeReference("cubeName", StringType, nullable = false)(),
+    Seq(AttributeReference("dbName", StringType, nullable = false)(),
+      AttributeReference("tableName", StringType, nullable = false)(),
       AttributeReference("isRegisteredWithSpark", BooleanType, nullable = false)())
   }
 }
@@ -161,7 +161,7 @@ case class ShowLoadsCommand(schemaNameOp: Option[String], cube: String, limit: O
 /**
  * Describe formatted for hive table
  */
-case class DescribeFormattedCommand(sql: String, tblIdentifier: Seq[String])
+case class DescribeFormattedCommand(sql: String, tblIdentifier: TableIdentifier)
   extends LogicalPlan with Command {
   override def children: Seq[LogicalPlan] = Seq.empty
 
@@ -181,16 +181,18 @@ case class CarbonDictionaryCatalystDecoder(
   override def output: Seq[Attribute] = child.output
 }
 
-abstract class CarbonProfile(attributes: Seq[Attribute]) extends Serializable{
+abstract class CarbonProfile(attributes: Seq[Attribute]) extends Serializable {
   def isEmpty: Boolean = attributes.isEmpty
 }
+
 case class IncludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attributes)
+
 case class ExcludeProfile(attributes: Seq[Attribute]) extends CarbonProfile(attributes)
 
 case class FakeCarbonCast(child: Literal, dataType: DataType)
   extends LeafExpression with CodegenFallback {
 
-  override def toString: String = s"FakeCarbonCast($child as ${dataType.simpleString})"
+  override def toString: String = s"FakeCarbonCast($child as ${ dataType.simpleString })"
 
   override def checkInputDataTypes(): TypeCheckResult = {
     TypeCheckResult.TypeCheckSuccess
@@ -255,7 +257,7 @@ object PhysicalOperation1 extends PredicateHelper {
         val (fields, filters, other, aliases, _, sortOrder, limit) = collectProjectsAndFilters(
           child)
 
-        var aggExps: Seq[AggregateExpression1] = Nil
+        var aggExps: Seq[AggregateExpression] = Nil
         aggregateExpressions.foreach(v => {
           val list = findAggreagateExpression(v)
           aggExps = aggExps ++ list
@@ -276,12 +278,12 @@ object PhysicalOperation1 extends PredicateHelper {
     }
   }
 
-  def findAggreagateExpression(expr: Expression): Seq[AggregateExpression1] = {
+  def findAggreagateExpression(expr: Expression): Seq[AggregateExpression] = {
     val exprList = expr match {
-      case d: AggregateExpression1 => d :: Nil
+      case d: AggregateExpression => d :: Nil
       case Alias(ref, name) => findAggreagateExpression(ref)
       case other =>
-        var listout: Seq[AggregateExpression1] = Nil
+        var listout: Seq[AggregateExpression] = Nil
 
         other.children.foreach(v => {
           val list = findAggreagateExpression(v)
@@ -317,7 +319,7 @@ object PhysicalOperation1 extends PredicateHelper {
           case Alias(ref, name) => ref
           case others => others
         }.filter {
-          case d: AggregateExpression1 => true
+          case d: AggregateExpression => true
           case _ => false
         }
         (fields, filters, other, aliases ++ collectAliases(aggregateExpressions), Some(
@@ -352,6 +354,28 @@ object PhysicalOperation1 extends PredicateHelper {
   }
 }
 
+case class PositionLiteral(expr: Expression, intermediateDataType: DataType)
+  extends LeafExpression with CodegenFallback {
+  override def dataType: DataType = expr.dataType
+
+  override def nullable: Boolean = false
+
+  type EvaluatedType = Any
+  var position = -1
+
+  def setPosition(pos: Int): Unit = position = pos
+
+  override def toString: String = s"PositionLiteral($position : $expr)"
+
+  override def eval(input: InternalRow): Any = {
+    if (position != -1) {
+      input.get(position, intermediateDataType)
+    } else {
+      expr.eval(input)
+    }
+  }
+}
+
 /**
  * Matches a logical aggregation that can be performed on distributed data in two steps.  The first
  * operates on the data in each partition performing partial aggregation for each group.  The second
@@ -367,85 +391,98 @@ object PhysicalOperation1 extends PredicateHelper {
  * - Partial aggregate expressions.
  * - Input to the aggregation.
  */
-object PartialAggregation {
-  type ReturnType =
-  (Seq[Attribute], Seq[NamedExpression], Seq[Expression], Seq[NamedExpression], LogicalPlan)
+object CarbonAggregation {
+  type ReturnType = (Seq[Expression], Seq[NamedExpression], LogicalPlan)
 
   private def convertAggregatesForPushdown(convertUnknown: Boolean,
-      rewrittenAggregateExpressions: Seq[Expression]) = {
-    var counter: Int = 0
-    var updatedExpressions = MutableList[Expression]()
-    rewrittenAggregateExpressions.foreach(v => {
-      val updated = convertAggregate(v, counter, convertUnknown)
-      updatedExpressions += updated
-      counter = counter + 1
-    })
-    updatedExpressions
+      rewrittenAggregateExpressions: Seq[Expression],
+      oneAttr: AttributeReference) = {
+    if (canBeConvertedToCarbonAggregate(rewrittenAggregateExpressions)) {
+      var counter: Int = 0
+      var updatedExpressions = MutableList[Expression]()
+      rewrittenAggregateExpressions.foreach(v => {
+        val updated = convertAggregate(v, counter, convertUnknown, oneAttr)
+        updatedExpressions += updated
+        counter = counter + 1
+      })
+      updatedExpressions
+    } else {
+      rewrittenAggregateExpressions
+    }
   }
 
-  def makePositionLiteral(expr: Expression, index: Int): PositionLiteral = {
-    val posLiteral = PositionLiteral(expr, MeasureAggregatorUDT)
+  def makePositionLiteral(expr: Expression, index: Int, dataType: DataType): PositionLiteral = {
+    val posLiteral = PositionLiteral(expr, dataType)
     posLiteral.setPosition(index)
     posLiteral
   }
 
-  def convertAggregate(current: Expression, index: Int, convertUnknown: Boolean): Expression = {
-    if (convertUnknown) {
+  def convertAggregate(current: Expression,
+      index: Int,
+      convertUnknown: Boolean,
+      oneAttr: AttributeReference): Expression = {
+    if (!convertUnknown && canBeConverted(current)) {
       current.transform {
-        case a@SumCarbon(_, _) => a
-        case a@AverageCarbon(_, _) => a
-        case a@MinCarbon(_, _) => a
-        case a@MaxCarbon(_, _) => a
-        case a@SumDistinctCarbon(_, _) => a
-        case a@CountDistinctCarbon(_) => a
-        case a@CountCarbon(_) => a
-        case anyAggr: AggregateExpression1 => anyAggr
+        case Average(attr: AttributeReference) =>
+          val convertedDataType = transformArrayType(attr)
+          CarbonAverage(makePositionLiteral(convertedDataType, index, convertedDataType.dataType))
+        case Average(Cast(attr: AttributeReference, dataType)) =>
+          val convertedDataType = transformArrayType(attr)
+          CarbonAverage(
+              makePositionLiteral(convertedDataType, index, convertedDataType.dataType))
+        case Count(Seq(s: Literal)) =>
+          CarbonCount(s, Some(makePositionLiteral(transformLongType(oneAttr), index, LongType)))
+        case Count(Seq(attr: AttributeReference)) =>
+          CarbonCount(makePositionLiteral(transformLongType(attr), index, LongType))
+        case Sum(attr: AttributeReference) =>
+          Sum(makePositionLiteral(attr, index, attr.dataType))
+        case Sum(Cast(attr: AttributeReference, dataType)) =>
+          Sum(Cast(makePositionLiteral(attr, index, attr.dataType), dataType))
+        case Min(attr: AttributeReference) => Min(makePositionLiteral(attr, index, attr.dataType))
+        case Min(Cast(attr: AttributeReference, dataType)) =>
+          Min(Cast(makePositionLiteral(attr, index, attr.dataType), dataType))
+        case Max(attr: AttributeReference) =>
+          Max(makePositionLiteral(attr, index, attr.dataType))
+        case Max(Cast(attr: AttributeReference, dataType)) =>
+          Max(Cast(makePositionLiteral(attr, index, attr.dataType), dataType))
       }
     } else {
-      current.transform {
-        case a@Sum(attr: AttributeReference) => SumCarbon(makePositionLiteral(attr, index))
-        case a@Sum(cast@Cast(attr: AttributeReference, _)) => SumCarbon(
-          makePositionLiteral(attr, index), cast.dataType)
-        case a@Average(attr: AttributeReference) => AverageCarbon(makePositionLiteral(attr, index))
-        case a@Average(cast@Cast(attr: AttributeReference, _)) => AverageCarbon(
-          makePositionLiteral(attr, index), cast.dataType)
-        case a@Min(attr: AttributeReference) => MinCarbon(makePositionLiteral(attr, index))
-        case a@Min(cast@Cast(attr: AttributeReference, _)) => MinCarbon(
-          makePositionLiteral(attr, index), cast.dataType)
-        case a@Max(attr: AttributeReference) => MaxCarbon(makePositionLiteral(attr, index))
-        case a@Max(cast@Cast(attr: AttributeReference, _)) => MaxCarbon(
-          makePositionLiteral(attr, index), cast.dataType)
-        case a@SumDistinct(attr: AttributeReference) => SumDistinctCarbon(
-          makePositionLiteral(attr, index))
-        case a@SumDistinct(cast@Cast(attr: AttributeReference, _)) => SumDistinctCarbon(
-          makePositionLiteral(attr, index), cast.dataType)
-        case a@CountDistinct(attr: AttributeReference) => CountDistinctCarbon(
-          makePositionLiteral(attr, index))
-        case a@CountDistinct(childSeq) if childSeq.size == 1 =>
-          childSeq.head match {
-            case attr: AttributeReference => CountDistinctCarbon(makePositionLiteral(attr, index))
-            case _ => a
-          }
-        case a@Count(s@Literal(_, _)) =>
-          CountCarbon(makePositionLiteral(s, index))
-        case a@Count(attr: AttributeReference) =>
-          if (attr.name.equals("*")) {
-            CountCarbon(makePositionLiteral(Literal("*"), index))
-          } else {
-            CountCarbon(makePositionLiteral(attr, index))
-          }
-      }
+      current
     }
   }
 
+  def canBeConverted(current: Expression): Boolean = current match {
+    case Alias(AggregateExpression(Average(attr: AttributeReference), _, false), _) => true
+    case Alias(AggregateExpression(Average(Cast(attr: AttributeReference, _)), _, false), _) => true
+    case Alias(AggregateExpression(Count(Seq(s: Literal)), _, false), _) => true
+    case Alias(AggregateExpression(Count(Seq(attr: AttributeReference)), _, false), _) => true
+    case Alias(AggregateExpression(Sum(attr: AttributeReference), _, false), _) => true
+    case Alias(AggregateExpression(Sum(Cast(attr: AttributeReference, _)), _, false), _) => true
+    case Alias(AggregateExpression(Min(attr: AttributeReference), _, false), _) => true
+    case Alias(AggregateExpression(Min(Cast(attr: AttributeReference, _)), _, false), _) => true
+    case Alias(AggregateExpression(Max(attr: AttributeReference), _, false), _) => true
+    case Alias(AggregateExpression(Max(Cast(attr: AttributeReference, _)), _, false), _) => true
+    case _ => false
+  }
+
+  def transformArrayType(attr: AttributeReference): AttributeReference = {
+    AttributeReference(attr.name, ArrayType(DoubleType), attr.nullable, attr.metadata)(attr.exprId,
+      attr.qualifiers)
+  }
+
+  def transformLongType(attr: AttributeReference): AttributeReference = {
+    AttributeReference(attr.name, LongType, attr.nullable, attr.metadata)(attr.exprId,
+      attr.qualifiers)
+  }
+
   /**
    * There should be sync between carbonOperators validation and here. we should not convert to
    * carbon aggregates if the validation does not satisfy.
    */
-  private def canBeConvertedToCarbonAggregate(expressions: Seq[Expression]): Boolean = {
+  def canBeConvertedToCarbonAggregate(expressions: Seq[Expression]): Boolean = {
     val detailQuery = expressions.map {
       case attr@AttributeReference(_, _, _, _) => true
-      case par: Alias if par.children.head.isInstanceOf[AggregateExpression1] => true
+      case Alias(agg: AggregateExpression, name) => true
       case _ => false
     }.exists(!_)
     !detailQuery
@@ -454,6 +491,7 @@ object PartialAggregation {
   def unapply(plan: LogicalPlan): Option[ReturnType] = unapply((plan, false))
 
   def unapply(combinedPlan: (LogicalPlan, Boolean)): Option[ReturnType] = {
+    val oneAttr = getOneAttribute(combinedPlan._1)
     combinedPlan._1 match {
       case Aggregate(groupingExpressions, aggregateExpressionsOrig, child) =>
 
@@ -463,99 +501,28 @@ object PartialAggregation {
             aggregateExpressionsOrig
           }
           else {
-            // First calculate partialComputation before converting and then check whether it could
-            // be converted or not. This type of checks are necessary for queries like
-            // select sum(col)+10 from table. Here the aggregates are different for
-            // partialComputation and aggregateExpressionsOrig. So first check on partialComputation
-            val preCheckEval = getPartialEvaluation(groupingExpressions, aggregateExpressionsOrig)
-            preCheckEval match {
-              case Some(allExprs) =>
-                if (canBeConvertedToCarbonAggregate(allExprs._1)) {
-                  convertAggregatesForPushdown(false, aggregateExpressionsOrig)
-                } else {
-                  aggregateExpressionsOrig
-                }
-              case _ => aggregateExpressionsOrig
-            }
+            convertAggregatesForPushdown(false, aggregateExpressionsOrig, oneAttr)
           }
-        val evaluation = getPartialEvaluation(groupingExpressions, aggregateExpressions)
-
-        evaluation match {
-          case(Some((partialComputation,
-              rewrittenAggregateExpressions,
-              namedGroupingAttributes))) =>
-            // Convert the other aggregations for push down to Carbon layer.
-            // Here don't touch earlier converted native carbon aggregators.
-            val convertedPartialComputation =
-              if (combinedPlan._2) {
-                partialComputation
-              }
-              else {
-                convertAggregatesForPushdown(true, partialComputation)
-                  .asInstanceOf[Seq[NamedExpression]]
-              }
-
-            Some(
-              (namedGroupingAttributes,
-                rewrittenAggregateExpressions,
-                groupingExpressions,
-                convertedPartialComputation,
-                child))
-          case _ => None
-        }
-
+        Some((groupingExpressions, aggregateExpressions.asInstanceOf[Seq[NamedExpression]], child))
       case _ => None
     }
   }
 
-  def getPartialEvaluation(groupingExpressions: Seq[Expression],
-      aggregateExpressions: Seq[Expression]):
-      Option[(Seq[NamedExpression], Seq[NamedExpression], Seq[Attribute])] = {
-    // Collect all aggregate expressions.
-    val allAggregates =
-      aggregateExpressions.flatMap(_ collect { case a: AggregateExpression1 => a })
-    // Collect all aggregate expressions that can be computed partially.
-    val partialAggregates =
-      aggregateExpressions.flatMap(_ collect { case p: PartialAggregate1 => p })
-
-    // Only do partial aggregation if supported by all aggregate expressions.
-    if (allAggregates.size == partialAggregates.size) {
-      // Create a map of expressions to their partial evaluations for all aggregate expressions.
-      val partialEvaluations: Map[TreeNodeRef, SplitEvaluation] =
-        partialAggregates.map(a => (new TreeNodeRef(a), a.asPartial)).toMap
-
-      // We need to pass all grouping expressions though so the grouping can happen a second
-      // time. However some of them might be unnamed so we alias them allowing them to be
-      // referenced in the second aggregation.
-      val namedGroupingExpressions: Map[Expression, NamedExpression] = groupingExpressions.map {
-        case n: NamedExpression => (n, n)
-        case other => (other, Alias(other, "PartialGroup")())
-      }.toMap
-
-      // Replace aggregations with a new expression that computes the result from the already
-      // computed partial evaluations and grouping values.
-      val rewrittenAggregateExpressions = aggregateExpressions.map(_.transformUp {
-        case e: Expression if partialEvaluations.contains(new TreeNodeRef(e)) =>
-          partialEvaluations(new TreeNodeRef(e)).finalEvaluation
-
-        case e: Expression =>
-          // Should trim aliases around `GetField`s. These aliases are introduced while
-          // resolving struct field accesses, because `GetField` is not a `NamedExpression`.
-          // (Should we just turn `GetField` into a `NamedExpression`?)
-          namedGroupingExpressions.collectFirst {
-            case (expr, ne) if expr semanticEquals e => ne.toAttribute
-          }.getOrElse(e)
-      }).asInstanceOf[Seq[NamedExpression]]
-
-      val partialComputation =
-        (namedGroupingExpressions.values ++
-         partialEvaluations.values.flatMap(_.partialEvaluations)).toSeq
-      val namedGroupingAttributes = namedGroupingExpressions.values.map(_.toAttribute).toSeq
-      Some(partialComputation, rewrittenAggregateExpressions, namedGroupingAttributes)
+  def getOneAttribute(plan: LogicalPlan): AttributeReference = {
+    var relation: LogicalRelation = null
+    plan collect {
+      case l: LogicalRelation => relation = l
+    }
+    if (relation != null) {
+      relation.output.find { p =>
+        p.dataType match {
+          case n: NumericType => true
+          case _ => false
+        }
+      }.getOrElse(relation.output.head)
     } else {
-      None
+      null
     }
-
   }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
index 79d8ffa..2bf50da 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
@@ -20,6 +20,7 @@ package org.apache.spark.sql
 import scala.language.implicitConversions
 
 import org.apache.spark.SparkContext
+import org.apache.spark.sql.catalyst.ParserDialect
 import org.apache.spark.sql.catalyst.analysis.{Analyzer, OverrideCatalog}
 import org.apache.spark.sql.catalyst.optimizer.{DefaultOptimizer, Optimizer}
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
@@ -52,7 +53,7 @@ class CarbonContext(val sc: SparkContext, val storePath: String) extends HiveCon
   override protected[sql] lazy val optimizer: Optimizer =
     new CarbonOptimizer(DefaultOptimizer, conf)
 
-  override protected[sql] def dialectClassName = classOf[CarbonSQLDialect].getCanonicalName
+  protected[sql] override def getSQLDialect(): ParserDialect = new CarbonSQLDialect(this)
 
   experimental.extraStrategies = CarbonStrategy.getStrategy(self)
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
index f95acf4..94b38a4 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
@@ -24,6 +24,7 @@ import scala.language.implicitConversions
 
 import org.apache.hadoop.fs.Path
 import org.apache.spark._
+import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.MultiInstanceRelation
 import org.apache.spark.sql.catalyst.expressions.AttributeReference
 import org.apache.spark.sql.catalyst.plans.logical._
@@ -56,7 +57,11 @@ class CarbonSource
       case _ =>
         val options = new CarbonOption(parameters)
         val tableIdentifier = options.tableIdentifier.split("""\.""").toSeq
-        CarbonDatasourceRelation(tableIdentifier, None)(sqlContext)
+        val ident = tableIdentifier match {
+          case Seq(name) => TableIdentifier(name)
+          case Seq(db, name) => TableIdentifier(name, Some(db))
+        }
+        CarbonDatasourceRelation(ident, None)(sqlContext)
     }
 
   }
@@ -120,14 +125,14 @@ class CarbonSource
  * This relation is stored to hive metastore
  */
 private[sql] case class CarbonDatasourceRelation(
-    tableIdentifier: Seq[String],
+    tableIdentifier: TableIdentifier,
     alias: Option[String])
   (@transient context: SQLContext)
   extends BaseRelation with Serializable with Logging {
 
   def carbonRelation: CarbonRelation = {
     CarbonEnv.getInstance(context)
-      .carbonCatalog.lookupRelation2(tableIdentifier, None)(sqlContext)
+      .carbonCatalog.lookupRelation1(tableIdentifier, None)(sqlContext)
       .asInstanceOf[CarbonRelation]
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/ead0076b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index a4ac246..600519f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -32,6 +32,7 @@ import org.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueI
 import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.carbondata.core.carbon.metadata.datatype.DataType
 import org.carbondata.core.carbon.metadata.encoder.Encoding
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 import org.carbondata.query.carbon.util.DataTypeUtil
 
 /**
@@ -62,7 +63,7 @@ case class CarbonDictionaryDecoder(
             !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
             canBeDecoded(attr)) {
           val newAttr = AttributeReference(a.name,
-            convertCarbonToSparkDataType(carbonDimension.getDataType),
+            convertCarbonToSparkDataType(carbonDimension),
             a.nullable,
             a.metadata)(a.exprId,
             a.qualifiers).asInstanceOf[Attribute]
@@ -88,8 +89,8 @@ case class CarbonDictionaryDecoder(
     }
   }
 
-  def convertCarbonToSparkDataType(dataType: DataType): types.DataType = {
-    dataType match {
+  def convertCarbonToSparkDataType(carbonDimension: CarbonDimension): types.DataType = {
+    carbonDimension.getDataType match {
       case DataType.STRING => StringType
       case DataType.INT => IntegerType
       case DataType.LONG => LongType
@@ -125,6 +126,9 @@ case class CarbonDictionaryDecoder(
     dictIds
   }
 
+
+  override def outputsUnsafeRows: Boolean = true
+
   override def doExecute(): RDD[InternalRow] = {
     attachTree(this, "execute") {
       val storePath = sqlContext.catalog.asInstanceOf[CarbonMetastoreCatalog].storePath
@@ -143,20 +147,21 @@ case class CarbonDictionaryDecoder(
           val dicts: Seq[Dictionary] = getDictionary(absoluteTableIdentifiers,
             forwardDictionaryCache)
           new Iterator[InternalRow] {
+            val unsafeProjection = UnsafeProjection.create(output.map(_.dataType).toArray)
             override final def hasNext: Boolean = iter.hasNext
 
             override final def next(): InternalRow = {
               val row: InternalRow = iter.next()
               val data = row.toSeq(dataTypes).toArray
               for (i <- data.indices) {
-                if (dicts(i) != null) {
+                if (dicts(i) != null && data(i) != null) {
                   data(i) = toType(DataTypeUtil
                     .getDataBasedOnDataType(dicts(i)
                       .getDictionaryValueForKey(data(i).asInstanceOf[Integer]),
                       getDictionaryColumnIds(i)._3))
                 }
               }
-              new GenericMutableRow(data)
+              unsafeProjection(new GenericMutableRow(data))
             }
           }
         }


[45/56] [abbrv] incubator-carbondata git commit: Add spark version print in example (#694)

Posted by jb...@apache.org.
Add spark version print in example (#694)

* add spark version print in example

* fix style


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

Branch: refs/heads/master
Commit: bf7557d18c48e1db9349bc03b5a9f576f5b46ec4
Parents: 8777640
Author: Jacky Li <ja...@huawei.com>
Authored: Mon Jun 20 17:30:01 2016 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Mon Jun 20 15:00:01 2016 +0530

----------------------------------------------------------------------
 .../scala/org/carbondata/examples/util/InitForExamples.scala   | 6 ++++++
 1 file changed, 6 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/bf7557d1/examples/src/main/scala/org/carbondata/examples/util/InitForExamples.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/carbondata/examples/util/InitForExamples.scala b/examples/src/main/scala/org/carbondata/examples/util/InitForExamples.scala
index 2d193c0..4fa4ce9 100644
--- a/examples/src/main/scala/org/carbondata/examples/util/InitForExamples.scala
+++ b/examples/src/main/scala/org/carbondata/examples/util/InitForExamples.scala
@@ -25,6 +25,8 @@ import org.apache.spark.sql.CarbonContext
 
 import org.carbondata.core.util.CarbonProperties
 
+// scalastyle:off println
+
 object InitForExamples {
 
   def currentPath: String = new File(this.getClass.getResource("/").getPath + "/../../")
@@ -39,6 +41,8 @@ object InitForExamples {
       .setMaster("local[2]"))
     sc.setLogLevel("ERROR")
 
+    println(s"Starting $appName using spark version ${sc.version}")
+
     val cc = new CarbonContext(sc, storeLocation)
     cc.setConf("carbon.kettle.home", kettleHome)
     cc.setConf("hive.metastore.warehouse.dir", hiveMetaPath)
@@ -51,3 +55,5 @@ object InitForExamples {
     cc
   }
 }
+// scalastyle:on println
+


[53/56] [abbrv] incubator-carbondata git commit: Merge pull request #710 from ravipesala/merge3

Posted by jb...@apache.org.
Merge pull request #710 from ravipesala/merge3

Fixed issue of dataframe query filters are not pushed down

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

Branch: refs/heads/master
Commit: d95742daf922165345f19abc406dad344fb2bd85
Parents: 3c41deb 8397673
Author: Liang Chen <ch...@huawei.com>
Authored: Tue Jun 21 17:46:42 2016 +0530
Committer: GitHub <no...@github.com>
Committed: Tue Jun 21 17:46:42 2016 +0530

----------------------------------------------------------------------
 .../scala/org/apache/spark/sql/optimizer/CarbonOptimizer.scala     | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------



[20/56] [abbrv] incubator-carbondata git commit: Refactored core package and fixed all testcases (#684)

Posted by jb...@apache.org.
Refactored core package and fixed all testcases (#684)

* Optimizing detail query

* Optimizing detail query flow

* Optimizing detail query flow

* Optimized raw detail query to improve push up performance.

* Fixed bugs

* reverted wrong check in

* Rebased the code

* Removed aggregation from core

* Refactored core package and fixed test cases

* Fixed bugs

* Fixed review comments and deleted aggregate classes after merge from master

* Removed unused code


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

Branch: refs/heads/master
Commit: 6288ec7152bbb832884de24e2e5912a0feecc0af
Parents: a83dba3
Author: Ravindra Pesala <ra...@gmail.com>
Authored: Sat Jun 18 06:47:19 2016 +0530
Committer: Jacky Li <ja...@huawei.com>
Committed: Sat Jun 18 09:17:19 2016 +0800

----------------------------------------------------------------------
 .../aggregator/CalculatedMeasureAggregator.java |  30 --
 .../aggregator/CustomMeasureAggregator.java     |  39 ---
 .../query/aggregator/MeasureAggregator.java     | 144 --------
 .../impl/AbstractMeasureAggregatorBasic.java    |  60 ----
 .../impl/AbstractMeasureAggregatorMaxMin.java   | 177 ----------
 .../query/aggregator/impl/BitSet.java           | 337 -------------------
 .../aggregator/impl/CustomAggregatorHelper.java | 235 -------------
 .../impl/avg/AbstractAvgAggregator.java         |  28 --
 .../impl/avg/AvgBigDecimalAggregator.java       | 218 ------------
 .../impl/avg/AvgDoubleAggregator.java           | 210 ------------
 .../aggregator/impl/avg/AvgLongAggregator.java  | 195 -----------
 .../aggregator/impl/count/CountAggregator.java  | 204 -----------
 ...bstractDistinctCountAggregatorObjectSet.java | 103 ------
 .../impl/distinct/DistinctCountAggregator.java  | 319 ------------------
 .../DistinctCountAggregatorObjectSet.java       |  96 ------
 ...tinctCountBigDecimalAggregatorObjectSet.java |  86 -----
 .../DistinctCountLongAggregatorObjectSet.java   |  89 -----
 .../distinct/DistinctStringCountAggregator.java | 165 ---------
 .../SumDistinctBigDecimalAggregator.java        | 233 -------------
 .../distinct/SumDistinctDoubleAggregator.java   | 231 -------------
 .../distinct/SumDistinctLongAggregator.java     | 221 ------------
 .../dummy/AbstractMeasureAggregatorDummy.java   |  70 ----
 .../impl/dummy/DummyBigDecimalAggregator.java   |  63 ----
 .../impl/dummy/DummyDoubleAggregator.java       |  64 ----
 .../impl/dummy/DummyLongAggregator.java         |  59 ----
 .../impl/max/AbstractMaxAggregator.java         |  61 ----
 .../aggregator/impl/max/MaxAggregator.java      |  68 ----
 .../impl/max/MaxBigDecimalAggregator.java       |  67 ----
 .../aggregator/impl/max/MaxLongAggregator.java  |  67 ----
 .../impl/min/AbstractMinAggregator.java         |  61 ----
 .../aggregator/impl/min/MinAggregator.java      |  66 ----
 .../impl/min/MinBigDecimalAggregator.java       |  66 ----
 .../aggregator/impl/min/MinLongAggregator.java  |  66 ----
 .../impl/sum/SumBigDecimalAggregator.java       | 179 ----------
 .../impl/sum/SumDoubleAggregator.java           | 179 ----------
 .../aggregator/impl/sum/SumLongAggregator.java  | 165 ---------
 .../util/MeasureAggregatorFactory.java          | 182 ----------
 .../query/carbon/aggregator/DataAggregator.java |  76 -----
 .../dimension/DimensionDataAggregator.java      |  40 ---
 .../impl/ColumnGroupDimensionsAggregator.java   | 109 ------
 .../DirectDictionaryDimensionAggregator.java    | 114 -------
 .../impl/FixedLengthDimensionAggregator.java    | 131 -------
 .../impl/VariableLengthDimensionAggregator.java | 126 -------
 .../expression/ExpressionAggregator.java        | 135 --------
 .../impl/ListBasedResultAggregator.java         | 128 ++++---
 .../impl/MapBasedResultAggregator.java          | 171 ----------
 .../measure/MeasureDataAggregator.java          |  48 ---
 .../measure/impl/FactTableAggregator.java       |  87 -----
 .../carbon/executor/QueryExecutorFactory.java   |  48 +--
 .../executor/impl/AbstractQueryExecutor.java    |  51 +--
 .../executor/impl/AggregationQueryExecutor.java |  47 ---
 .../executor/impl/CountStarQueryExecutor.java   |  45 ---
 .../executor/impl/DetailQueryExecutor.java      |   5 +-
 .../impl/DetailRawRecordQueryExcecutor.java     |  26 --
 .../impl/DetailRawRecordQueryExecutor.java      |  26 ++
 .../impl/DetailWithOrderByQueryExecutor.java    |  53 ---
 .../executor/impl/FunctionQueryExecutor.java    |  37 --
 .../executor/impl/QueryExecutorProperties.java  |  12 +-
 .../carbon/executor/infos/AggregatorInfo.java   |  30 +-
 .../executor/infos/BlockExecutionInfo.java      |  56 +--
 .../impl/InternalAbstractQueryExecutor.java     |  12 +-
 .../impl/InternalAggregationQueryExecutor.java  |  44 ---
 .../impl/InternalCountStartQueryExecutor.java   |  83 -----
 .../impl/InternalDetailQueryExecutor.java       |   7 +-
 .../InternalDetailWithOrderQueryExecutor.java   |  74 ----
 .../impl/InternalFunctionQueryExecutor.java     |  36 --
 .../executor/internal/impl/QueryRunner.java     |   7 +-
 .../query/carbon/executor/util/QueryUtil.java   | 189 +----------
 .../merger/AbstractScannedResultMerger.java     |   7 +-
 .../merger/impl/SortedScannedResultMerger.java  | 176 ----------
 .../query/carbon/model/CarbonQueryPlan.java     |  30 --
 .../carbon/model/CustomAggregateExpression.java | 149 --------
 .../query/carbon/model/QueryModel.java          |  55 ---
 .../processor/AbstractDataBlockProcessor.java   |  11 +-
 .../carbon/result/AbstractScannedResult.java    |   8 +
 .../query/carbon/result/BatchRawResult.java     | 130 ++-----
 .../query/carbon/result/BatchResult.java        |  64 ++--
 .../carbon/result/ListBasedResultWrapper.java   |   7 +-
 .../carbondata/query/carbon/result/Result.java  |   7 +-
 .../query/carbon/result/RowResult.java          |  45 ---
 .../carbon/result/impl/ListBasedResult.java     |  11 +-
 .../carbon/result/impl/MapBasedResult.java      | 141 --------
 .../AbstractDetailQueryResultIterator.java      |   8 +-
 .../ChunkBasedDetailResultIterator.java         |  75 +++++
 .../iterator/ChunkBasedResultIterator.java      |  71 ----
 .../result/iterator/ChunkRawRowIterartor.java   |  56 ---
 .../result/iterator/ChunkRowIterator.java       |   5 +-
 .../iterator/DetailQueryResultIterator.java     |  12 +-
 .../iterator/DetailRawQueryResultIterator.java  |  89 +++--
 .../preparator/QueryResultPreparator.java       |   5 +-
 .../impl/AbstractQueryResultPreparator.java     |  61 ++--
 .../impl/DetailQueryResultPreparatorImpl.java   | 139 ++++++++
 .../impl/QueryResultPreparatorImpl.java         | 297 ----------------
 .../impl/RawQueryResultPreparatorImpl.java      |  87 ++---
 .../executer/RowLevelFilterExecuterImpl.java    |  59 ++--
 .../MeasureColumnResolvedFilterInfo.java        |  10 -
 .../query/scanner/impl/CarbonKey.java           | 105 ------
 .../query/scanner/impl/CarbonValue.java         |  87 -----
 .../carbondata/hadoop/CarbonRecordReader.java   |   8 +-
 .../spark/sql/CarbonCatalystOperators.scala     | 330 ------------------
 .../spark/sql/CarbonDictionaryDecoder.scala     |  29 +-
 .../org/apache/spark/sql/CarbonOperators.scala  |  35 +-
 .../org/apache/spark/sql/CarbonSQLConf.scala    |  11 -
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  10 +-
 .../execution/command/carbonTableSchema.scala   |  33 +-
 .../spark/sql/hive/CarbonStrategies.scala       |  86 ++---
 .../CarbonDecoderOptimizerHelper.scala          |   8 +-
 .../spark/sql/optimizer/CarbonOptimizer.scala   |  39 ++-
 .../org/carbondata/spark/CarbonFilters.scala    |  58 ++--
 .../scala/org/carbondata/spark/KeyVal.scala     |  28 +-
 .../spark/rdd/CarbonCleanFilesRDD.scala         |  22 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  40 +--
 .../spark/rdd/CarbonDeleteLoadRDD.scala         |  40 +--
 .../spark/rdd/CarbonDropAggregateTableRDD.scala |  85 -----
 .../spark/rdd/CarbonDropCubeRDD.scala           |  77 -----
 .../spark/rdd/CarbonDropTableRDD.scala          |  76 +++++
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |   3 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |  42 +--
 .../carbondata/spark/util/QueryPlanUtil.scala   |   5 +-
 .../AllDataTypesTestCaseAggregate.scala         |   1 +
 .../store/colgroup/ColGroupMinMaxTest.java      |   2 +-
 121 files changed, 877 insertions(+), 9154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/CalculatedMeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/CalculatedMeasureAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/CalculatedMeasureAggregator.java
deleted file mode 100644
index d115a91..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/CalculatedMeasureAggregator.java
+++ /dev/null
@@ -1,30 +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.carbondata.query.aggregator;
-
-public interface CalculatedMeasureAggregator extends MeasureAggregator {
-
-  /**
-   * Calculate calculated measures
-   *
-   * @param aggregators
-   */
-  void calculateCalcMeasure(MeasureAggregator[] aggregators);
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/CustomMeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/CustomMeasureAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/CustomMeasureAggregator.java
deleted file mode 100644
index 7c14a7c..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/CustomMeasureAggregator.java
+++ /dev/null
@@ -1,39 +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.carbondata.query.aggregator;
-
-import java.util.List;
-
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.ColumnExpression;
-
-public interface CustomMeasureAggregator extends MeasureAggregator {
-  /**
-   * Aggregate method with generic row interface where RowIntf holds value for
-   * each column given in MeasureAggregator@getColumns()
-   */
-  void agg(RowIntf row);
-
-  /**
-   * @return List of columns required for the aggregator
-   */
-  List<ColumnExpression> getColumns();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/MeasureAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/MeasureAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/MeasureAggregator.java
deleted file mode 100644
index 515d307..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/MeasureAggregator.java
+++ /dev/null
@@ -1,144 +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.carbondata.query.aggregator;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.Serializable;
-import java.math.BigDecimal;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-
-/**
- * Class Description : MeasureAggregator interface. It will be implemented by
- * all the aggregator functions eg: sum, avg, max, min, etc, will be used for
- * aggregate the measure value based on kind of aggregator
- */
-
-public interface MeasureAggregator extends Serializable, Comparable<MeasureAggregator> {
-
-  /**
-   * Below method will be used to aggregate the Double value
-   *
-   * @param newVal
-   */
-  void agg(double newVal);
-
-  /**
-   * Below method will be used to aggregate the object value
-   *
-   * @param newVal
-   */
-  void agg(Object newVal);
-
-  /**
-   * Below method will be used to aggregate the value based on index
-   *
-   * @param newVal
-   * @param index
-   */
-  void agg(MeasureColumnDataChunk newVal, int index);
-
-  /**
-   * Get the Serialize byte array
-   *
-   * @return
-   */
-  byte[] getByteArray();
-
-  /**
-   * This method will be used to set the new value
-   *
-   * @param newValue
-   */
-  void setNewValue(Object newValue);
-
-  /**
-   * This method return the object value of the MeasureAggregator
-   *
-   * @return aggregated value
-   */
-  Object getValueObject();
-
-  /**
-   * This method return the object value of the MeasureAggregator
-   *
-   * @return aggregated value
-   */
-  Double getDoubleValue();
-
-  /**
-   * This method return the object value of the MeasureAggregator
-   *
-   * @return aggregated value
-   */
-  Long getLongValue();
-
-  BigDecimal getBigDecimalValue();
-
-  /**
-   * This method merge the aggregated value based on aggregator passed
-   *
-   * @param aggregator type of aggregator
-   */
-  void merge(MeasureAggregator aggregator);
-
-  /**
-   * Is first time. It means it was never used for aggregating any value.
-   *
-   * @return
-   */
-  boolean isFirstTime();
-
-  /**
-   * it creates the new copy of MeasureAggregator
-   *
-   * @return MeasureAggregator
-   */
-  MeasureAggregator getCopy();
-
-  /**
-   * Write the state of the class to buffer
-   */
-  void writeData(DataOutput output) throws IOException;
-
-  /**
-   * Read the state of the class and set to the object
-   */
-  void readData(DataInput inPut) throws IOException;
-
-  MeasureAggregator get();
-
-  /**
-   * Merge the byte arrays
-   *
-   * @param value
-   */
-  void merge(byte[] value);
-
-  /**
-   * Below method will be used to get the
-   * new instance
-   *
-   * @return new instance
-   */
-  MeasureAggregator getNew();
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorBasic.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorBasic.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorBasic.java
deleted file mode 100644
index a4ce00b..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorBasic.java
+++ /dev/null
@@ -1,60 +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.carbondata.query.aggregator.impl;
-
-import java.math.BigDecimal;
-
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * AbstractMeasureAggregatorSum
- * Used for custom Carbon Aggregator sum
- */
-public abstract class AbstractMeasureAggregatorBasic implements MeasureAggregator {
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 1L;
-
-  protected boolean firstTime = true;
-
-  @Override public void agg(double newVal) {
-  }
-
-  @Override public Double getDoubleValue() {
-    return null;
-  }
-
-  @Override public Long getLongValue() {
-    return null;
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java b/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java
deleted file mode 100644
index 99a8ed9..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/AbstractMeasureAggregatorMaxMin.java
+++ /dev/null
@@ -1,177 +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.carbondata.query.aggregator.impl;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.io.ObjectInput;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutput;
-import java.io.ObjectOutputStream;
-import java.math.BigDecimal;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.max.MaxAggregator;
-
-/**
- * AbstractMeasureAggregatorMaxMin
- * Used for custom Carbon Aggregator max  min
- */
-public abstract class AbstractMeasureAggregatorMaxMin implements MeasureAggregator {
-  private static final long serialVersionUID = 1L;
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(MaxAggregator.class.getName());
-
-  public Comparable<Object> aggVal;
-
-  public boolean firstTime = true;
-
-  protected abstract void internalAgg(Object value);
-
-  @Override public void agg(double newVal) {
-    internalAgg((Double) newVal);
-    firstTime = false;
-  }
-
-  @Override public Double getDoubleValue() {
-    return (Double) ((Object) aggVal);
-  }
-
-  @Override public void agg(Object newVal) {
-    internalAgg(newVal);
-    firstTime = false;
-  }
-
-  @Override public Long getLongValue() {
-    return (Long) ((Object) aggVal);
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return (BigDecimal) ((Object) aggVal);
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-  }
-
-  /**
-   * This method return the max value as an object
-   * a8
-   *
-   * @return max value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  @Override public boolean isFirstTime() {
-    return firstTime;
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public int compareTo(MeasureAggregator msrAggr) {
-    @SuppressWarnings("unchecked") Comparable<Object> other =
-        (Comparable<Object>) msrAggr.getValueObject();
-
-    return aggVal.compareTo(other);
-  }
-
-  @Override public void writeData(DataOutput dataOutput) throws IOException {
-    ByteArrayOutputStream bos = null;
-    ObjectOutput out = null;
-
-    try {
-      dataOutput.writeBoolean(firstTime);
-      bos = new ByteArrayOutputStream();
-      out = new ObjectOutputStream(bos);
-      out.writeObject(aggVal);
-      byte[] objectBytes = bos.toByteArray();
-      dataOutput.write(objectBytes.length);
-      dataOutput.write(objectBytes, 0, objectBytes.length);
-    } catch (Exception e) {
-      LOGGER.error(e,
-          "Problem while getting byte array in maxMinAggregator: " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(bos);
-    }
-  }
-
-  @SuppressWarnings("unchecked")
-  @Override
-  public void readData(DataInput inPut)
-      throws IOException {
-    ByteArrayInputStream bis = null;
-    ObjectInput in = null;
-    try {
-      int length = inPut.readInt();
-      firstTime = inPut.readBoolean();
-      byte[] data = new byte[length];
-      bis = new ByteArrayInputStream(data);
-      in = new ObjectInputStream(bis);
-      aggVal = (Comparable<Object>) in.readObject();
-    } catch (Exception e) {
-      LOGGER.error(e,
-          "Problem while getting byte array in maxMinAggregator: " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(bis);
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    byte[] objectBytes = new byte[0];
-    if (firstTime) {
-      return objectBytes;
-    }
-    ObjectOutput out = null;
-    ByteArrayOutputStream bos = null;
-    try {
-      bos = new ByteArrayOutputStream();
-      out = new ObjectOutputStream(bos);
-      out.writeObject(aggVal);
-      objectBytes = bos.toByteArray();
-    } catch (IOException e) {
-      LOGGER.error(e,
-          "Problem while getting byte array in maxMinAggregator: " + e.getMessage());
-    } finally {
-      CarbonUtil.closeStreams(bos);
-    }
-    return objectBytes;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/BitSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/BitSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/BitSet.java
deleted file mode 100644
index 61fe179..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/BitSet.java
+++ /dev/null
@@ -1,337 +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.carbondata.query.aggregator.impl;
-
-import java.nio.ByteBuffer;
-import java.nio.ByteOrder;
-import java.util.Arrays;
-
-/**
- * This class implements a vector of bits that grows as needed. Each component
- * of the bit set has a {@code boolean} value. The bits of a {@code BitSet} are
- * indexed by nonnegative integers. Individual indexed bits can be examined,
- * set, or cleared. One {@code BitSet} may be used to modify the contents of
- * another {@code BitSet} through logical AND, logical inclusive OR, and logical
- * exclusive OR operations.
- * By default, all bits in the set initially have the value {@code false}.
- * Every bit set has a current size, which is the number of bits of space
- * currently in use by the bit set. Note that the size is related to the
- * implementation of a bit set, so it may change with implementation. The length
- * of a bit set relates to logical length of a bit set and is defined
- * independently of implementation.
- * Unless otherwise noted, passing a null parameter to any of the methods in a
- * {@code BitSet} will result in a {@code NullPointerException}.
- * A {@code BitSet} is not safe for multithreaded use without external
- * synchronization.
- *
- * @author Arthur van Hoff
- * @author Michael McCloskey
- * @author Martin Buchholz
- * @since JDK1.0
- */
-public class BitSet implements java.io.Serializable {
-  /* use serialVersionUID from JDK 1.0.2 for interoperability */
-  private static final long serialVersionUID = 7997698588986878753L;
-
-  /*
-   * BitSets are packed into arrays of "words." Currently a word is a long,
-   * which consists of 64 bits, requiring 6 address bits. The choice of word
-   * size is determined purely by performance concerns.
-   */
-  private static final int ADDRESS_BITS_PER_WORD = 6;
-
-  private static final int BITS_PER_WORD = 1 << ADDRESS_BITS_PER_WORD;
-
-  /* Used to shift left or right for a partial word mask */
-  private static final long WORD_MASK = 0xffffffffffffffffL;
-
-  /**
-   * The internal field corresponding to the serialField "bits".
-   */
-  private long[] words;
-
-  /**
-   * The number of words in the logical size of this BitSet.
-   */
-  private transient int wordsInUse;
-
-  /**
-   * Creates a new bit set. All bits are initially {@code false}.
-   */
-  public BitSet() {
-    initWords(BITS_PER_WORD);
-  }
-
-  /**
-   * Creates a bit set using words as the internal representation. The last
-   * word (if there is one) must be non-zero.
-   */
-  private BitSet(long[] words) {
-    this.words = words;
-    this.wordsInUse = words.length;
-    checkInvariants();
-  }
-
-  /**
-   * Given a bit index, return word index containing it.
-   */
-  private static int wordIndex(int bitIndex) {
-    return bitIndex >> ADDRESS_BITS_PER_WORD;
-  }
-
-  /**
-   * Returns a new bit set containing all the bits in the given byte array.
-   * More precisely, <br>
-   * {@code BitSet.valueOf(bytes).get(n) == ((bytes[n/8] & (1<<(n%8))) != 0)}
-   * <br>
-   * for all {@code n <  8 * bytes.length}.
-   * This method is equivalent to
-   * {@code BitSet.valueOf(ByteBuffer.wrap(bytes))}.
-   *
-   * @param bytes a byte array containing a little-endian representation of a
-   *              sequence of bits to be used as the initial bits of the new bit
-   *              set
-   * @since 1.7
-   */
-  public static BitSet valueOf(byte[] bytes) {
-    return BitSet.valueOf(ByteBuffer.wrap(bytes));
-  }
-
-  /**
-   * Returns a new bit set containing all the bits in the given byte buffer
-   * between its position and limit.
-   * More precisely, <br>
-   * {@code BitSet.valueOf(bb).get(n) == ((bb.get(bb.position()+n/8) & (1<<(n%8))) != 0)}
-   * <br>
-   * for all {@code n < 8 * bb.remaining()}.
-   * The byte buffer is not modified by this method, and no reference to the
-   * buffer is retained by the bit set.
-   *
-   * @param bb a byte buffer containing a little-endian representation of a
-   *           sequence of bits between its position and limit, to be used as
-   *           the initial bits of the new bit set
-   * @since 1.7
-   */
-  public static BitSet valueOf(ByteBuffer bb) {
-    bb = bb.slice().order(ByteOrder.LITTLE_ENDIAN);
-    int n = 0;//CHECKSTYLE:OFF
-    for (n = bb.remaining(); ; n--) {//CHECKSTYLE:ON
-      if (n > 0 && bb.get(n - 1) == 0) {
-        continue;
-      } else {
-        break;
-      }
-    }
-    long[] words = new long[(n + 7) / 8];
-    bb.limit(n);
-    int i = 0;//CHECKSTYLE:OFF
-    while (bb.remaining() >= 8) {//CHECKSTYLE:ON
-      words[i++] = bb.getLong();
-    }
-
-    int j = 0;
-    for (int remaining = bb.remaining(); j < remaining; j++) {
-      words[i] |= (bb.get() & 0xffL) << (8 * j);
-    }
-    return new BitSet(words);
-  }
-
-  /**
-   * Every public method must preserve these invariants.
-   */
-  private void checkInvariants() {
-    assert(wordsInUse == 0 || words[wordsInUse - 1] != 0);
-    assert(wordsInUse >= 0 && wordsInUse <= words.length);
-    assert(wordsInUse == words.length || words[wordsInUse] == 0);
-  }
-
-  private void initWords(int nbits) {
-    words = new long[wordIndex(nbits - 1) + 1];
-  }
-
-  /**
-   * Returns a new byte array containing all the bits in this bit set.
-   * More precisely, if <br>
-   * {@code byte[] bytes = s.toByteArray();} <br>
-   * then {@code bytes.length == (s.length()+7)/8} and <br>
-   * {@code s.get(n) == ((bytes[n/8] & (1<<(n%8))) != 0)} <br>
-   * for all {@code n < 8 * bytes.length}.
-   *
-   * @return a byte array containing a little-endian representation of all the
-   * bits in this bit set
-   * @since 1.7
-   */
-  public byte[] toByteArray() {
-    int n = wordsInUse;
-    if (n == 0) {
-      return new byte[0];
-    }
-    int len = 8 * (n - 1);
-    for (long x = words[n - 1]; x != 0; x >>>= 8) {
-      len++;
-    }
-    byte[] bytes = new byte[len];
-    ByteBuffer bb = ByteBuffer.wrap(bytes).order(ByteOrder.LITTLE_ENDIAN);
-    for (int i = 0; i < n - 1; i++) {
-      bb.putLong(words[i]);
-    }
-    for (long x = words[n - 1]; x != 0; x >>>= 8) {
-      bb.put((byte) (x & 0xff));
-    }
-    return bytes;
-  }
-
-  /**
-   * Ensures that the BitSet can hold enough words.
-   *
-   * @param wordsRequired the minimum acceptable number of words.
-   */
-  private void ensureCapacity(int wordsRequired) {
-    if (words.length < wordsRequired) {
-      // Allocate larger of doubled size or required size
-      int request = Math.max(2 * words.length, wordsRequired);
-      words = Arrays.copyOf(words, request);
-    }
-  }
-
-  /**
-   * Ensures that the BitSet can accommodate a given wordIndex, temporarily
-   * violating the invariants. The caller must restore the invariants before
-   * returning to the user, possibly using recalculateWordsInUse().
-   *
-   * @param wordIndex the index to be accommodated.
-   */
-  private void expandTo(int wordIndex) {
-    int wordsRequired = wordIndex + 1;
-    if (wordsInUse < wordsRequired) {
-      ensureCapacity(wordsRequired);
-      wordsInUse = wordsRequired;
-    }
-  }
-
-  /**
-   * Sets the bit at the specified index to {@code true}.
-   *
-   * @param bitIndex a bit index
-   * @throws IndexOutOfBoundsException if the specified index is negative
-   * @since JDK1.0
-   */
-  public void set(int bitIndex) {
-    if (bitIndex < 0) {
-      throw new IndexOutOfBoundsException("bitIndex < 0: " + bitIndex);
-    }
-    int wordIndex = wordIndex(bitIndex);
-    expandTo(wordIndex);
-
-    words[wordIndex] |= (1L << bitIndex); // Restores invariants
-
-    checkInvariants();
-  }
-
-  /**
-   * Returns the index of the first bit that is set to {@code true} that
-   * occurs on or after the specified starting index. If no such bit exists
-   * then {@code -1} is returned.
-   * To iterate over the {@code true} bits in a {@code BitSet}, use the
-   * following loop:
-   * <pre>
-   * {@code
-   * for (int i = bs.nextSetBit(0); i >= 0; i = bs.nextSetBit(i+1)) {
-   *     // operate on index i here
-   * }}
-   * </pre>
-   *
-   * @param fromIndex the index to start checking from (inclusive)
-   * @return the index of the next set bit, or {@code -1} if there is no such
-   * bit
-   * @throws IndexOutOfBoundsException if the specified index is negative
-   * @since 1.4
-   */
-  public int nextSetBit(int fromIndex) {
-    if (fromIndex < 0) {
-      throw new IndexOutOfBoundsException("fromIndex < 0: " + fromIndex);
-    }
-    checkInvariants();
-
-    int u = wordIndex(fromIndex);
-    if (u >= wordsInUse) {
-      return -1;
-    }
-    long word = words[u] & (WORD_MASK << fromIndex);
-
-    while (true) {
-      if (word != 0) {
-        return (u * BITS_PER_WORD) + Long.numberOfTrailingZeros(word);
-      }
-
-      u++;
-      if (u == wordsInUse) {
-        return -1;
-      }
-      word = words[u];
-    }
-  }
-
-  /**
-   * Returns the number of bits set to {@code true} in this {@code BitSet}.
-   *
-   * @return the number of bits set to {@code true} in this {@code BitSet}
-   * @since 1.4
-   */
-  public int cardinality() {
-    int sum = 0;
-    for (int i = 0; i < wordsInUse; i++) {
-      sum += Long.bitCount(words[i]);
-    }
-    return sum;
-  }
-
-  /**
-   * Performs a logical <b>OR</b> of this bit set with the bit set argument.
-   * This bit set is modified so that a bit in it has the value {@code true}
-   * if and only if it either already had the value {@code true} or the
-   * corresponding bit in the bit set argument has the value {@code true}.
-   *
-   * @param set a bit set
-   */
-  public void or(BitSet set) {
-    if (this == set) {
-      return;
-    }
-    int wordsInCommon = Math.min(wordsInUse, set.wordsInUse);
-
-    if (wordsInUse < set.wordsInUse) {
-      ensureCapacity(set.wordsInUse);
-      wordsInUse = set.wordsInUse;
-    }
-
-    // Perform logical OR on words in common
-    for (int i = 0; i < wordsInCommon; i++) {
-      words[i] |= set.words[i];
-    }
-    // Copy any remaining words
-    if (wordsInCommon < set.wordsInUse) {
-      System.arraycopy(set.words, wordsInCommon, words, wordsInCommon, wordsInUse - wordsInCommon);
-    }
-    // recalculateWordsInUse() is unnecessary
-    checkInvariants();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/CustomAggregatorHelper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/CustomAggregatorHelper.java b/core/src/main/java/org/carbondata/query/aggregator/impl/CustomAggregatorHelper.java
deleted file mode 100644
index 0e23df1..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/CustomAggregatorHelper.java
+++ /dev/null
@@ -1,235 +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.carbondata.query.aggregator.impl;
-
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileInputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.nio.charset.Charset;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.core.util.CarbonUtil;
-
-import org.apache.commons.codec.binary.Base64;
-
-public class CustomAggregatorHelper {
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CustomAggregatorHelper.class.getName());
-
-  /**
-   * surrogateKeyMap
-   */
-  private Map<String, Map<Integer, String>> surrogateKeyMap;
-
-  /**
-   * loadFolderList
-   */
-  private List<File> loadFolderList;
-
-  public CustomAggregatorHelper() {
-    surrogateKeyMap =
-        new HashMap<String, Map<Integer, String>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    loadFolderList = new ArrayList<File>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * Below method will be used to get the file list
-   *
-   * @param baseStorePath
-   * @param fileNameSearchPattern
-   * @return
-   */
-  private static File[] getFilesArray(File baseStorePath, final String fileNameSearchPattern) {
-    File[] listFiles = baseStorePath.listFiles(new FileFilter() {
-      @Override public boolean accept(File pathname) {
-        if (pathname.getName().indexOf(fileNameSearchPattern) > -1) {
-          return true;
-        }
-        return false;
-      }
-    });
-    return listFiles;
-  }
-
-  /**
-   * Below method will be used to get the member
-   *
-   * @param tableName
-   * @param columnName
-   * @param key
-   * @param cubeName
-   * @param schemaName
-   * @return member
-   */
-  public String getDimValue(String tableName, String columnName, int key, String cubeName,
-      String schemaName) {
-    Map<Integer, String> memberCache = surrogateKeyMap.get(tableName + '_' + columnName);
-    if (null == memberCache) {
-      loadLevelFile(tableName, columnName, cubeName, schemaName);
-    }
-    memberCache = surrogateKeyMap.get(tableName + '_' + columnName);
-    return memberCache.get(key);
-  }
-
-  /**
-   * Below method will be used to fill the level cache
-   *
-   * @param tableName
-   * @param columnName
-   * @param cubeName
-   * @param schemaName
-   */
-  private void loadLevelFile(String tableName, String columnName, String cubeName,
-      String schemaName) {
-    String baseLocation = CarbonUtil.getCarbonStorePath(schemaName, cubeName);
-    baseLocation = baseLocation + File.separator + schemaName + File.separator + cubeName;
-    if (loadFolderList.size() == 0) {
-      checkAndUpdateFolderList(baseLocation);
-    }
-    try {
-      File[] filesArray = null;
-      for (File loadFoler : loadFolderList) {
-        filesArray = getFilesArray(loadFoler, tableName + '_' + columnName);
-        for (int i = 0; i < filesArray.length; i++) {
-          readLevelFileAndUpdateCache(filesArray[i], tableName + '_' + columnName);
-        }
-      }
-    } catch (IOException e) {
-      LOGGER
-          .error("Problem while populating the cache");
-    }
-  }
-
-  /**
-   * Below method will be used to read the level files
-   *
-   * @param memberFile
-   * @param fileName
-   * @throws IOException
-   */
-  private void readLevelFileAndUpdateCache(File memberFile, String fileName) throws IOException {
-    FileInputStream fos = null;
-    FileChannel fileChannel = null;
-    try {
-      // create an object of FileOutputStream
-      fos = new FileInputStream(memberFile);
-
-      fileChannel = fos.getChannel();
-      Map<Integer, String> memberMap = surrogateKeyMap.get(fileName);
-
-      if (null == memberMap) {
-        memberMap = new HashMap<Integer, String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        surrogateKeyMap.put(fileName, memberMap);
-      }
-
-      long size = fileChannel.size();
-      int maxKey = 0;
-      ByteBuffer rowlengthToRead = null;
-      int len = 0;
-      ByteBuffer row = null;
-      int toread = 0;
-      byte[] bb = null;
-      String value = null;
-      int surrogateValue = 0;
-
-      boolean enableEncoding = Boolean.valueOf(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.ENABLE_BASE64_ENCODING,
-              CarbonCommonConstants.ENABLE_BASE64_ENCODING_DEFAULT));
-
-      while (fileChannel.position() < size) {
-        rowlengthToRead = ByteBuffer.allocate(4);
-        fileChannel.read(rowlengthToRead);
-        rowlengthToRead.rewind();
-        len = rowlengthToRead.getInt();
-        if (len == 0) {
-          continue;
-        }
-
-        row = ByteBuffer.allocate(len);
-        fileChannel.read(row);
-        row.rewind();
-        toread = row.getInt();
-        bb = new byte[toread];
-        row.get(bb);
-
-        if (enableEncoding) {
-          value = new String(Base64.decodeBase64(bb), Charset.defaultCharset());
-        } else {
-          value = new String(bb, Charset.defaultCharset());
-        }
-
-        surrogateValue = row.getInt();
-        memberMap.put(surrogateValue, value);
-
-        // check if max key is less than Surrogate key then update the max key
-        if (maxKey < surrogateValue) {
-          maxKey = surrogateValue;
-        }
-      }
-
-    } finally {
-      CarbonUtil.closeStreams(fileChannel, fos);
-    }
-  }
-
-  /**
-   * This method recursively checks the folder with Load_ inside each and
-   * every RS_x/TableName/Load_x and add in the folder list the load folders.
-   *
-   * @param baseStorePath
-   * @return
-   */
-  private File[] checkAndUpdateFolderList(String baseStorePath) {
-    File folders = new File(baseStorePath);
-    //
-    File[] rsFolders = folders.listFiles(new FileFilter() {
-      @Override public boolean accept(File pathname) {
-        boolean check = false;
-        check = pathname.isDirectory()
-            && pathname.getAbsolutePath().indexOf(CarbonCommonConstants.LOAD_FOLDER) > -1;
-        if (check) {
-          return true;
-        } else {
-          File[] checkFolder = checkAndUpdateFolderList(pathname.getAbsolutePath());
-          if (null != checkFolder) {
-            for (File f : checkFolder) {
-              loadFolderList.add(f);
-            }
-          }
-        }
-        return false;
-      }
-    });
-    return rsFolders;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java
deleted file mode 100644
index 24eff08..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AbstractAvgAggregator.java
+++ /dev/null
@@ -1,28 +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.carbondata.query.aggregator.impl.avg;
-
-import org.carbondata.query.aggregator.impl.AbstractMeasureAggregatorBasic;
-
-public abstract class AbstractAvgAggregator extends AbstractMeasureAggregatorBasic {
-
-  public abstract Object[] getAvgState();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java
deleted file mode 100644
index c6ba55d..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgBigDecimalAggregator.java
+++ /dev/null
@@ -1,218 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.aggregator.impl.avg;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class AvgBigDecimalAggregator extends AbstractAvgAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 5463736686281089871L;
-
-  /**
-   * total number of aggregate values
-   */
-  protected double count;
-
-  /**
-   * aggregate value
-   */
-  protected BigDecimal aggVal;
-
-  public AvgBigDecimalAggregator() {
-    aggVal = new BigDecimal(0);
-  }
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        byte[] valueByte = new byte[buffer.getInt()];
-        buffer.get(valueByte);
-        BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-        aggVal = aggVal.add(valueBigDecimal);
-
-        count += buffer.getDouble();
-        firstTime = false;
-      }
-      return;
-    }
-
-    if (firstTime) {
-      aggVal = (BigDecimal) newVal;
-      firstTime = false;
-    } else {
-      aggVal = aggVal.add((BigDecimal) newVal);
-    }
-    count++;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      BigDecimal value = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-      aggVal = aggVal.add(value);
-      firstTime = false;
-      count++;
-    }
-  }
-
-  @Override public Object[] getAvgState() {
-    return new Object[]{aggVal, count};
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    byte[] bytes = DataTypeUtil.bigDecimalToByte(aggVal);
-    ByteBuffer allocate =
-        ByteBuffer.allocate(4 + bytes.length + CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    allocate.putInt(bytes.length);
-    allocate.put(bytes);
-    allocate.putDouble(count);
-    allocate.rewind();
-
-    return allocate.array();
-  }
-
-  /**
-   * Return the average of the aggregate values
-   *
-   * @return average aggregate value
-   */
-  @Override public BigDecimal getBigDecimalValue() {
-    return aggVal.divide(new BigDecimal(count), 6);
-  }
-
-  /**
-   * This method merge the aggregated value, in average aggregator it will add
-   * count and aggregate value
-   *
-   * @param aggregator Avg Aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    AvgBigDecimalAggregator avgAggregator = (AvgBigDecimalAggregator) aggregator;
-    if (!avgAggregator.isFirstTime()) {
-      aggVal = aggVal.add(avgAggregator.aggVal);
-      count += avgAggregator.count;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * This method return the average value as an object
-   *
-   * @return average value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal.divide(new BigDecimal(count));
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (BigDecimal) newValue;
-    count = 1;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeUTF(aggVal.toString());
-    output.writeDouble(count);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = new BigDecimal(inPut.readUTF());
-    count = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    AvgBigDecimalAggregator avg = new AvgBigDecimalAggregator();
-    avg.aggVal = aggVal;
-    avg.count = count;
-    avg.firstTime = firstTime;
-    return avg;
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    BigDecimal val = getBigDecimalValue();
-    BigDecimal otherVal = o.getBigDecimalValue();
-
-    return val.compareTo(otherVal);
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof AvgBigDecimalAggregator)) {
-      return false;
-    }
-    AvgBigDecimalAggregator o = (AvgBigDecimalAggregator) obj;
-    return getBigDecimalValue().equals(o.getBigDecimalValue());
-  }
-
-  @Override public int hashCode() {
-    return getBigDecimalValue().hashCode();
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-
-    byte[] valueByte = new byte[buffer.getInt()];
-    buffer.get(valueByte);
-    BigDecimal valueBigDecimal = DataTypeUtil.byteToBigDecimal(valueByte);
-    aggVal = aggVal.add(valueBigDecimal);
-    count += buffer.getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return (aggVal.divide(new BigDecimal(count))) + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new AvgBigDecimalAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java
deleted file mode 100644
index bacff29..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgDoubleAggregator.java
+++ /dev/null
@@ -1,210 +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.carbondata.query.aggregator.impl.avg;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class AvgDoubleAggregator extends AbstractAvgAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 5463736686281089871L;
-
-  /**
-   * total number of aggregate values
-   */
-  protected double count;
-
-  /**
-   * aggregate value
-   */
-  protected double aggVal;
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(double newVal) {
-    aggVal += newVal;
-    count++;
-    firstTime = false;
-  }
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        aggVal += buffer.getDouble();
-        count += buffer.getDouble();
-        firstTime = false;
-      }
-      return;
-    }
-    aggVal += ((Number) newVal).doubleValue();
-    count++;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal += dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      count++;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(2 * CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putDouble(aggVal);
-    buffer.putDouble(count);
-    return buffer.array();
-  }
-
-  /**
-   * Return the average of the aggregate values
-   *
-   * @return average aggregate value
-   */
-  @Override public Double getDoubleValue() {
-    return aggVal / count;
-  }
-
-  @Override public Object[] getAvgState() {
-    return new Object[]{aggVal, count};
-  }
-
-  /**
-   * This method merge the aggregated value, in average aggregator it will add
-   * count and aggregate value
-   *
-   * @param aggregator Avg Aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    AvgDoubleAggregator avgAggregator = (AvgDoubleAggregator) aggregator;
-    if (!avgAggregator.isFirstTime()) {
-      aggVal += avgAggregator.aggVal;
-      count += avgAggregator.count;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * This method return the average value as an object
-   *
-   * @return average value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal / count;
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Double) newValue;
-    count = 1;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeDouble(aggVal);
-    output.writeDouble(count);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readDouble();
-    count = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    AvgDoubleAggregator avg = new AvgDoubleAggregator();
-    avg.aggVal = aggVal;
-    avg.count = count;
-    avg.firstTime = firstTime;
-    return avg;
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    double val = getDoubleValue();
-    double otherVal = o.getDoubleValue();
-    if (val > otherVal) {
-      return 1;
-    }
-    if (val < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof AvgDoubleAggregator)) {
-      return false;
-    }
-    AvgDoubleAggregator o = (AvgDoubleAggregator)obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    aggVal += buffer.getDouble();
-    count += buffer.getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return (aggVal / count) + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new AvgDoubleAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java
deleted file mode 100644
index 6290de6..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/avg/AvgLongAggregator.java
+++ /dev/null
@@ -1,195 +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.carbondata.query.aggregator.impl.avg;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class AvgLongAggregator extends AbstractAvgAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 5463736686281089871L;
-
-  /**
-   * total number of aggregate values
-   */
-  protected double count;
-
-  /**
-   * aggregate value
-   */
-  protected long aggVal;
-
-  /**
-   * Average Aggregate function which will add all the aggregate values and it
-   * will increment the total count every time, for average value
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      ByteBuffer buffer = ByteBuffer.wrap((byte[]) newVal);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        aggVal += buffer.getLong();
-        count += buffer.getDouble();
-        firstTime = false;
-      }
-      return;
-    }
-    aggVal += (Long) newVal;
-    count++;
-    firstTime = false;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal += dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-      count++;
-      firstTime = false;
-    }
-  }
-
-  @Override public Object[] getAvgState() {
-    return new Object[]{aggVal, count};
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (firstTime) {
-      return new byte[0];
-    }
-    ByteBuffer buffer = ByteBuffer.allocate(
-        CarbonCommonConstants.LONG_SIZE_IN_BYTE + CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putLong(aggVal);
-    buffer.putDouble(count);
-    return buffer.array();
-  }
-
-  @Override public Long getLongValue() {
-    return aggVal / (long) count;
-  }
-
-  /**
-   * This method merge the aggregated value, in average aggregator it will add
-   * count and aggregate value
-   *
-   * @param aggregator Avg Aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    AvgLongAggregator avgAggregator = (AvgLongAggregator) aggregator;
-    if (!avgAggregator.isFirstTime()) {
-      aggVal += avgAggregator.aggVal;
-      count += avgAggregator.count;
-      firstTime = false;
-    }
-  }
-
-  /**
-   * This method return the average value as an object
-   *
-   * @return average value as an object
-   */
-  @Override public Object getValueObject() {
-    return aggVal / count;
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal = (Long) newValue;
-    count = 1;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeBoolean(firstTime);
-    output.writeLong(aggVal);
-    output.writeDouble(count);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    firstTime = inPut.readBoolean();
-    aggVal = inPut.readLong();
-    count = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    AvgLongAggregator avg = new AvgLongAggregator();
-    avg.aggVal = aggVal;
-    avg.count = count;
-    avg.firstTime = firstTime;
-    return avg;
-  }
-
-  @Override public int compareTo(MeasureAggregator o) {
-    long val = getLongValue();
-    long otherVal = o.getLongValue();
-    if (val > otherVal) {
-      return 1;
-    } else if (val < otherVal) {
-      return -1;
-    } else {
-      return 0;
-    }
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof AvgLongAggregator)) {
-      return false;
-    }
-    AvgLongAggregator o = (AvgLongAggregator)obj;
-    return getLongValue().equals(o.getLongValue());
-  }
-
-  @Override public int hashCode() {
-    return getLongValue().hashCode();
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    aggVal += buffer.getLong();
-    count += buffer.getDouble();
-    firstTime = false;
-  }
-
-  public String toString() {
-    return (aggVal / count) + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new AvgLongAggregator();
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java
deleted file mode 100644
index 352e7aa..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/count/CountAggregator.java
+++ /dev/null
@@ -1,204 +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.carbondata.query.aggregator.impl.count;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-/**
- * Class Description : It will return total count of values
- */
-public class CountAggregator implements MeasureAggregator {
-
-  /**
-   * serialVersionUID
-   */
-  private static final long serialVersionUID = 2678878935295306313L;
-
-  /**
-   * aggregate value
-   */
-  private double aggVal;
-
-  /**
-   * Count Aggregate function which update the total count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(double newVal) {
-    aggVal++;
-  }
-
-  /**
-   * Count Aggregate function which update the total count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    aggVal++;
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      aggVal++;
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    ByteBuffer buffer = ByteBuffer.allocate(CarbonCommonConstants.DOUBLE_SIZE_IN_BYTE);
-    buffer.putDouble(aggVal);
-    return buffer.array();
-  }
-
-  /**
-   * Returns the total count
-   *
-   * @return total count
-   */
-  @Override public Double getDoubleValue() {
-    return aggVal;
-  }
-
-  @Override public Long getLongValue() {
-    return (long) aggVal;
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return new BigDecimal(aggVal);
-  }
-
-  /**
-   * Merge the total count with the aggregator
-   *
-   * @param aggregator count aggregator
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    CountAggregator countAggregator = (CountAggregator) aggregator;
-    aggVal += countAggregator.aggVal;
-  }
-
-  /**
-   * Overloaded Aggregate function will be used for Aggregate tables because
-   * aggregate table will have fact_count as a measure. It will update the
-   * total count
-   *
-   * @param newVal
-   *            new value
-   * @param factCount
-   *            total fact count
-   *
-   */
-  //    @Override
-  //    public void agg(double newVal, double factCount)
-  //    {
-  //        agg(newVal, null, 0, 0);
-  //    }
-
-  /**
-   * This method return the count value as an object
-   *
-   * @return count value as an object
-   */
-
-  @Override public Object getValueObject() {
-    return aggVal;
-  }
-
-  /**
-   * @see MeasureAggregator#setNewValue(Object)
-   */
-  @Override public void setNewValue(Object newValue) {
-    aggVal += Double.parseDouble(String.valueOf(newValue));
-  }
-
-  @Override public boolean isFirstTime() {
-    return false;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-    output.writeDouble(aggVal);
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    aggVal = inPut.readDouble();
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    CountAggregator aggregator = new CountAggregator();
-    aggregator.aggVal = aggVal;
-    return aggregator;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    aggVal += buffer.getDouble();
-  }
-
-  @Override public int compareTo(MeasureAggregator obj) {
-    double val = getDoubleValue();
-    double otherVal = obj.getDoubleValue();
-    if (val > otherVal) {
-      return 1;
-    }
-    if (val < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof CountAggregator)) {
-      return false;
-    }
-    CountAggregator o = (CountAggregator)obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  public String toString() {
-    return aggVal + "";
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new CountAggregator();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java
deleted file mode 100644
index 0629007..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/AbstractDistinctCountAggregatorObjectSet.java
+++ /dev/null
@@ -1,103 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.io.DataInput;
-import java.io.DataOutput;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public abstract class AbstractDistinctCountAggregatorObjectSet implements MeasureAggregator {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  protected Set<Object> valueSetForObj;
-
-  public AbstractDistinctCountAggregatorObjectSet() {
-    valueSetForObj = new HashSet<Object>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-  }
-
-  /**
-   * Distinct count Aggregate function which update the Distinct count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    valueSetForObj.add(newVal);
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    return null;
-  }
-
-  @Override public Double getDoubleValue() {
-    return (double) valueSetForObj.size();
-  }
-
-  @Override public Long getLongValue() {
-    return (long) valueSetForObj.size();
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    return new BigDecimal(valueSetForObj.size());
-  }
-
-  @Override public Object getValueObject() {
-    return valueSetForObj.size();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    valueSetForObj.add(newValue);
-  }
-
-  @Override public boolean isFirstTime() {
-    return false;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-
-  }
-
-  public String toString() {
-    return valueSetForObj.size() + "";
-  }
-
-  @Override public void merge(byte[] value) {
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java
deleted file mode 100644
index 1b2b33d..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregator.java
+++ /dev/null
@@ -1,319 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.DataInput;
-import java.io.DataInputStream;
-import java.io.DataOutput;
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.math.BigDecimal;
-import java.nio.ByteBuffer;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-import org.roaringbitmap.IntIterator;
-import org.roaringbitmap.RoaringBitmap;
-
-/**
- * * The distinct count aggregator
- * Ex:
- * ID NAME Sales
- * <p>1 a 200
- * <p>2 a 100
- * <p>3 a 200
- * select count(distinct sales) # would result 2
- * select count(sales) # would result 3
- */
-public class DistinctCountAggregator implements MeasureAggregator {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DistinctCountAggregator.class.getName());
-  /**
-   *
-   */
-  private static final long serialVersionUID = 6313463368629960186L;
-  /**
-   * For Spark CARBON to avoid heavy object transfer it better to flatten
-   * the Aggregators. There is no aggregation expected after setting this value.
-   */
-  private Double computedFixedValue;
-  /**
-   *
-   */
-  //    private Set<Double> valueSet;
-  private RoaringBitmap valueSet;
-
-  private byte[] data;
-
-  private double minValue;
-
-  public DistinctCountAggregator(Object minValue) {
-    valueSet = new RoaringBitmap();
-    if (minValue instanceof BigDecimal) {
-      this.minValue = ((BigDecimal) minValue).doubleValue();
-    } else if (minValue instanceof Long) {
-      this.minValue = ((Long) minValue).doubleValue();
-    } else {
-      this.minValue = (Double) minValue;
-    }
-  }
-
-  public DistinctCountAggregator() {
-    valueSet = new RoaringBitmap();
-  }
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-    valueSet.add((int) (newVal - minValue));
-  }
-
-  /**
-   * Distinct count Aggregate function which update the Distinct count
-   *
-   * @param newVal new value
-   */
-  @Override public void agg(Object newVal) {
-    if (newVal instanceof byte[]) {
-      byte[] values = (byte[]) newVal;
-      ByteBuffer buffer = ByteBuffer.wrap(values);
-      buffer.rewind();
-      while (buffer.hasRemaining()) {
-        valueSet.add(buffer.getInt());
-      }
-      return;
-    } else {
-      double value = new Double(newVal.toString());
-      agg(value);
-    }
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSet.add((int) dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-    }
-  }
-
-  /**
-   * Below method will be used to get the value byte array
-   */
-  @Override public byte[] getByteArray() {
-    if (valueSet.getCardinality() == 0) {
-      return new byte[0];
-    }
-    IntIterator iterator = valueSet.getIntIterator();
-    ByteBuffer buffer = ByteBuffer.allocate(valueSet.getCardinality() * 4 + 8);
-    buffer.putDouble(minValue);
-    while (iterator.hasNext()) {
-      buffer.putInt(iterator.next());
-    }
-    buffer.rewind();
-    return buffer.array();
-  }
-
-  private void agg(RoaringBitmap set2, double minValue) {
-    if (this.minValue == minValue) {
-      valueSet.or(set2);
-    } else {
-      if (this.minValue > minValue) {
-        IntIterator intIterator = valueSet.getIntIterator();
-        while (intIterator.hasNext()) {
-          set2.add((int) ((double) (intIterator.next() + this.minValue) - minValue));
-        }
-        this.minValue = minValue;
-        this.valueSet = set2;
-      } else {
-        IntIterator intIterator = set2.getIntIterator();
-        while (intIterator.hasNext()) {
-          valueSet.add((int) ((double) (intIterator.next() + minValue) - this.minValue));
-        }
-      }
-    }
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountAggregator distinctCountAggregator = (DistinctCountAggregator) aggregator;
-    readData();
-    distinctCountAggregator.readData();
-    if (distinctCountAggregator.valueSet != null) {
-      agg(distinctCountAggregator.valueSet, distinctCountAggregator.minValue);
-    }
-  }
-
-  @Override public Double getDoubleValue() {
-    if (computedFixedValue == null) {
-      readData();
-      return (double) valueSet.getCardinality();
-    }
-    return computedFixedValue;
-  }
-
-  @Override public Long getLongValue() {
-    if (computedFixedValue == null) {
-      readData();
-      return (long) valueSet.getCardinality();
-    }
-    return computedFixedValue.longValue();
-  }
-
-  @Override public BigDecimal getBigDecimalValue() {
-    if (computedFixedValue == null) {
-      readData();
-      return new BigDecimal(valueSet.getCardinality());
-    }
-    return new BigDecimal(computedFixedValue);
-  }
-
-  @Override public Object getValueObject() {
-    return valueSet.getCardinality();
-  }
-
-  @Override public void setNewValue(Object newValue) {
-    computedFixedValue = (Double) newValue;
-    valueSet = null;
-  }
-
-  @Override public boolean isFirstTime() {
-    return false;
-  }
-
-  @Override public void writeData(DataOutput output) throws IOException {
-
-    if (computedFixedValue != null) {
-      ByteBuffer byteBuffer = ByteBuffer.allocate(4 + 8);
-      byteBuffer.putInt(-1);
-      byteBuffer.putDouble(computedFixedValue);
-      byteBuffer.flip();
-      output.write(byteBuffer.array());
-    } else {
-      if (valueSet != null) {
-        valueSet.serialize(output);
-      } else {
-        output.write(data);
-      }
-    }
-  }
-
-  @Override public void readData(DataInput inPut) throws IOException {
-    valueSet = new RoaringBitmap();
-    valueSet.deserialize(inPut);
-  }
-
-  private void readData() {
-    if (data != null && (valueSet == null || valueSet.isEmpty())) {
-      ByteArrayInputStream stream = new ByteArrayInputStream(data);
-      DataInputStream outputStream = new DataInputStream(stream);
-      try {
-        readData(outputStream);
-        outputStream.close();
-        data = null;
-      } catch (IOException e) {
-        LOGGER.error(e, e.getMessage());
-      }
-    }
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountAggregator aggr = new DistinctCountAggregator(minValue);
-    aggr.valueSet = valueSet.clone();
-    return aggr;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    double compFixedVal = getDoubleValue();
-    double otherVal = measureAggr.getDoubleValue();
-    if (compFixedVal > otherVal) {
-      return 1;
-    }
-    if (compFixedVal < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if(!(obj instanceof DistinctCountAggregator)) {
-      return false;
-    }
-    DistinctCountAggregator o = (DistinctCountAggregator) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
-    DataOutputStream outputStream = new DataOutputStream(byteStream);
-    try {
-      writeData(outputStream);
-    } catch (IOException ex) {
-      LOGGER.error(ex, ex.getMessage());
-    }
-    data = byteStream.toByteArray();
-    valueSet = null;
-    return this;
-  }
-
-  public String toString() {
-    if (computedFixedValue == null) {
-      readData();
-      return valueSet.getCardinality() + "";
-    }
-    return computedFixedValue + "";
-  }
-
-  public RoaringBitmap getBitMap() {
-    return valueSet;
-  }
-
-  public double getMinValue() {
-    return minValue;
-  }
-
-  @Override public void merge(byte[] value) {
-    if (0 == value.length) {
-      return;
-    }
-    ByteBuffer buffer = ByteBuffer.wrap(value);
-    buffer.rewind();
-    double currentMinValue = buffer.getDouble();
-    while (buffer.hasRemaining()) {
-      agg(buffer.getInt() + currentMinValue);
-    }
-  }
-
-  @Override public MeasureAggregator getNew() {
-    // TODO Auto-generated method stub
-    return new DistinctCountAggregator();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.java b/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.java
deleted file mode 100644
index 3b26e53..0000000
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/distinct/DistinctCountAggregatorObjectSet.java
+++ /dev/null
@@ -1,96 +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.carbondata.query.aggregator.impl.distinct;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.aggregator.MeasureAggregator;
-
-public class DistinctCountAggregatorObjectSet extends AbstractDistinctCountAggregatorObjectSet {
-
-  private static final long serialVersionUID = 6313463368629960186L;
-
-  /**
-   * just need to add the unique values to agg set
-   */
-  @Override public void agg(double newVal) {
-    valueSetForObj.add(newVal);
-  }
-
-  @Override public void agg(MeasureColumnDataChunk dataChunk, int index) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      valueSetForObj.add(dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index));
-    }
-  }
-
-  private void agg(Set<Object> set2) {
-    valueSetForObj.addAll(set2);
-  }
-
-  /**
-   * merge the valueset so that we get the count of unique values
-   */
-  @Override public void merge(MeasureAggregator aggregator) {
-    DistinctCountAggregatorObjectSet distinctCountAggregator =
-        (DistinctCountAggregatorObjectSet) aggregator;
-    agg(distinctCountAggregator.valueSetForObj);
-  }
-
-  @Override public MeasureAggregator getCopy() {
-    DistinctCountAggregatorObjectSet aggregator = new DistinctCountAggregatorObjectSet();
-    aggregator.valueSetForObj = new HashSet<Object>(valueSetForObj);
-    return aggregator;
-  }
-
-  @Override public int compareTo(MeasureAggregator measureAggr) {
-    double valueSetForObjSize = getDoubleValue();
-    double otherVal = measureAggr.getDoubleValue();
-    if (valueSetForObjSize > otherVal) {
-      return 1;
-    }
-    if (valueSetForObjSize < otherVal) {
-      return -1;
-    }
-    return 0;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof DistinctCountAggregatorObjectSet)) {
-      return false;
-    }
-    DistinctCountAggregatorObjectSet o = (DistinctCountAggregatorObjectSet) obj;
-    return getDoubleValue().equals(o.getDoubleValue());
-  }
-
-  @Override public int hashCode() {
-    return getDoubleValue().hashCode();
-  }
-
-  @Override public MeasureAggregator get() {
-    return this;
-  }
-
-  @Override public MeasureAggregator getNew() {
-    return new DistinctCountAggregatorObjectSet();
-  }
-
-}



[27/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
new file mode 100644
index 0000000..d09bab5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@ -0,0 +1,233 @@
+/*
+ * 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.carbondata.scan.filter.resolver;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.SortedMap;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.ByteUtil;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.conditional.BinaryConditionalExpression;
+import org.carbondata.scan.expression.logical.BinaryLogicalExpression;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.intf.FilterExecuterType;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
+
+public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverImpl {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 6629319265336666789L;
+
+  private List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList;
+  private List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
+  private AbsoluteTableIdentifier tableIdentifier;
+
+  public RowLevelRangeFilterResolverImpl(Expression exp, boolean isExpressionResolve,
+      boolean isIncludeFilter, AbsoluteTableIdentifier tableIdentifier) {
+    super(exp, isExpressionResolve, isIncludeFilter);
+    dimColEvaluatorInfoList =
+        new ArrayList<DimColumnResolvedFilterInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(
+        CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.tableIdentifier = tableIdentifier;
+  }
+
+  /**
+   * This method will return the filter values which is present in the range leve
+   * conditional expressions.
+   *
+   * @return
+   */
+  public byte[][] getFilterRangeValues() {
+    List<byte[]> filterValuesList = new ArrayList<byte[]>();
+    if (null != dimColEvaluatorInfoList.get(0).getFilterValues()) {
+      filterValuesList =
+          dimColEvaluatorInfoList.get(0).getFilterValues().getNoDictionaryFilterValuesList();
+      return filterValuesList.toArray((new byte[filterValuesList.size()][]));
+    }
+    return filterValuesList.toArray((new byte[filterValuesList.size()][]));
+
+  }
+
+  /**
+   * method will get the start key based on the filter surrogates
+   *
+   * @return start IndexKey
+   */
+  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
+      SortedMap<Integer, byte[]> noDictStartKeys) {
+    if (null == dimColEvaluatorInfoList.get(0).getStarIndexKey()) {
+      FilterUtil
+          .getStartKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
+              noDictStartKeys);
+    }
+  }
+
+  /**
+   * method will get the start key based on the filter surrogates
+   *
+   * @return end IndexKey
+   */
+  @Override public void getEndKey(SegmentProperties segmentProperties,
+      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
+      SortedMap<Integer, byte[]> noDicEndKeys) {
+    if (null == dimColEvaluatorInfoList.get(0).getEndIndexKey()) {
+      try {
+        FilterUtil.getEndKey(dimColEvaluatorInfoList.get(0).getDimensionResolvedFilterInstance(),
+            absoluteTableIdentifier, endKeys, segmentProperties);
+        FilterUtil
+            .getEndKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
+                noDicEndKeys);
+      } catch (QueryExecutionException e) {
+        // TODO Auto-generated catch block
+        e.printStackTrace();
+      }
+    }
+  }
+
+  private List<byte[]> getNoDictionaryRangeValues() {
+    List<ExpressionResult> listOfExpressionResults = new ArrayList<ExpressionResult>(20);
+    if (this.getFilterExpression() instanceof BinaryConditionalExpression) {
+      listOfExpressionResults =
+          ((BinaryConditionalExpression) this.getFilterExpression()).getLiterals();
+    }
+    List<byte[]> filterValuesList = new ArrayList<byte[]>(20);
+    for (ExpressionResult result : listOfExpressionResults) {
+      if (result.getString() == null) {
+        filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL.getBytes());
+        continue;
+      }
+      filterValuesList.add(result.getString().getBytes());
+    }
+    Comparator<byte[]> filterNoDictValueComaparator = new Comparator<byte[]>() {
+      @Override public int compare(byte[] filterMember1, byte[] filterMember2) {
+        return ByteUtil.UnsafeComparer.INSTANCE.compareTo(filterMember1, filterMember2);
+      }
+
+    };
+    Collections.sort(filterValuesList, filterNoDictValueComaparator);
+    return filterValuesList;
+  }
+
+  /**
+   * Method which will resolve the filter expression by converting the filter
+   * member to its assigned dictionary values.
+   */
+  public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
+    DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = null;
+    MeasureColumnResolvedFilterInfo msrColumnEvalutorInfo = null;
+    int index = 0;
+    if (exp instanceof BinaryLogicalExpression) {
+      BinaryLogicalExpression conditionalExpression = (BinaryLogicalExpression) exp;
+      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
+      for (ColumnExpression columnExpression : columnList) {
+        if (columnExpression.isDimension()) {
+          dimColumnEvaluatorInfo = new DimColumnResolvedFilterInfo();
+          DimColumnFilterInfo filterInfo = new DimColumnFilterInfo();
+          dimColumnEvaluatorInfo.setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
+          //dimColumnEvaluatorInfo.se
+          dimColumnEvaluatorInfo.setRowIndex(index++);
+          dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
+          dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
+          filterInfo.setFilterListForNoDictionaryCols(getNoDictionaryRangeValues());
+          filterInfo.setIncludeFilter(isIncludeFilter);
+          dimColumnEvaluatorInfo.setFilterValues(filterInfo);
+          dimColumnEvaluatorInfo
+              .addDimensionResolvedFilterInstance(columnExpression.getDimension(), filterInfo);
+          dimColEvaluatorInfoList.add(dimColumnEvaluatorInfo);
+        } else {
+          msrColumnEvalutorInfo = new MeasureColumnResolvedFilterInfo();
+          msrColumnEvalutorInfo.setRowIndex(index++);
+          msrColumnEvalutorInfo.setAggregator(
+              ((CarbonMeasure) columnExpression.getCarbonColumn()).getAggregateFunction());
+          msrColumnEvalutorInfo
+              .setColumnIndex(((CarbonMeasure) columnExpression.getCarbonColumn()).getOrdinal());
+          msrColumnEvalutorInfo.setType(columnExpression.getCarbonColumn().getDataType());
+          msrColEvalutorInfoList.add(msrColumnEvalutorInfo);
+        }
+      }
+    }
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which consists
+   * the mapping of the respective dimension and its surrogates involved in
+   * filter expression.
+   *
+   * @return DimColumnResolvedFilterInfo
+   */
+  public List<DimColumnResolvedFilterInfo> getDimColEvaluatorInfoList() {
+    return dimColEvaluatorInfoList;
+  }
+
+  /**
+   * Method will return the DimColumnResolvedFilterInfo instance which containts
+   * measure level details.
+   *
+   * @return MeasureColumnResolvedFilterInfo
+   */
+  public List<MeasureColumnResolvedFilterInfo> getMsrColEvalutorInfoList() {
+    return msrColEvalutorInfoList;
+  }
+
+  public AbsoluteTableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public Expression getFilterExpression() {
+    return this.exp;
+  }
+
+  /**
+   * This method will provide the executer type to the callee inorder to identify
+   * the executer type for the filter resolution, Row level filter executer is a
+   * special executer since it get all the rows of the specified filter dimension
+   * and will be send to the spark for processing
+   */
+  public FilterExecuterType getFilterExecuterType() {
+    switch (exp.getFilterExpressionType()) {
+      case GREATERTHAN:
+        return FilterExecuterType.ROWLEVEL_GREATERTHAN;
+      case GREATERTHAN_EQUALTO:
+        return FilterExecuterType.ROWLEVEL_GREATERTHAN_EQUALTO;
+      case LESSTHAN:
+        return FilterExecuterType.ROWLEVEL_LESSTHAN;
+      case LESSTHAN_EQUALTO:
+        return FilterExecuterType.ROWLEVEL_LESSTHAN_EQUALTO;
+
+      default:
+        return FilterExecuterType.ROWLEVEL;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java b/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
new file mode 100644
index 0000000..c684f5f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/metadata/FilterResolverMetadata.java
@@ -0,0 +1,62 @@
+/*
+ * 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.carbondata.scan.filter.resolver.metadata;
+
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+
+public class FilterResolverMetadata {
+  private AbsoluteTableIdentifier tableIdentifier;
+  private Expression expression;
+  private ColumnExpression columnExpression;
+  private boolean isIncludeFilter;
+
+  public AbsoluteTableIdentifier getTableIdentifier() {
+    return tableIdentifier;
+  }
+
+  public void setTableIdentifier(AbsoluteTableIdentifier tableIdentifier) {
+    this.tableIdentifier = tableIdentifier;
+  }
+
+  public Expression getExpression() {
+    return expression;
+  }
+
+  public void setExpression(Expression expression) {
+    this.expression = expression;
+  }
+
+  public ColumnExpression getColumnExpression() {
+    return columnExpression;
+  }
+
+  public void setColumnExpression(ColumnExpression columnExpression) {
+    this.columnExpression = columnExpression;
+  }
+
+  public boolean isIncludeFilter() {
+    return isIncludeFilter;
+  }
+
+  public void setIncludeFilter(boolean isIncludeFilter) {
+    this.isIncludeFilter = isIncludeFilter;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
new file mode 100644
index 0000000..ad7942a
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/DimColumnResolvedFilterInfo.java
@@ -0,0 +1,206 @@
+/*
+ * 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.carbondata.scan.filter.resolver.resolverinfo;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.carbondata.core.carbon.datastore.IndexKey;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.GenericQueryType;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.visitable.ResolvedFilterInfoVisitable;
+import org.carbondata.scan.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
+
+public class DimColumnResolvedFilterInfo implements Serializable, ResolvedFilterInfoVisitable {
+  /**
+   *
+   */
+  private static final long serialVersionUID = 3428115141211084114L;
+
+  /**
+   * column index in file
+   */
+  private int columnIndex = -1;
+
+  /**
+   * need compressed data from file
+   */
+  private boolean needCompressedData;
+
+  /**
+   * rowIndex
+   */
+  private int rowIndex = -1;
+
+  private boolean isDimensionExistsInCurrentSilce = true;
+
+  private int rsSurrogates;
+
+  private String defaultValue;
+
+  private transient Map<Integer, GenericQueryType> complexTypesWithBlockStartIndex;
+
+  private CarbonDimension dimension;
+
+  /**
+   * start index key of the block based on the keygenerator
+   */
+  private transient IndexKey starIndexKey;
+
+  /**
+   * end index key  which is been formed considering the max surrogate values
+   * from dictionary cache
+   */
+  private transient IndexKey endIndexKey;
+
+  /**
+   * reolved filter object of a particlar filter Expression.
+   */
+  private DimColumnFilterInfo resolvedFilterValueObj;
+
+  private Map<CarbonDimension, List<DimColumnFilterInfo>> dimensionResolvedFilter;
+
+  public DimColumnResolvedFilterInfo() {
+    dimensionResolvedFilter = new HashMap<CarbonDimension, List<DimColumnFilterInfo>>(20);
+  }
+
+  public IndexKey getStarIndexKey() {
+    return starIndexKey;
+  }
+
+  public void setStarIndexKey(IndexKey starIndexKey) {
+    this.starIndexKey = starIndexKey;
+  }
+
+  public IndexKey getEndIndexKey() {
+    return endIndexKey;
+  }
+
+  public void setEndIndexKey(IndexKey endIndexKey) {
+    this.endIndexKey = endIndexKey;
+  }
+
+  public void addDimensionResolvedFilterInstance(CarbonDimension dimension,
+      DimColumnFilterInfo filterResolvedObj) {
+    List<DimColumnFilterInfo> currentVals = dimensionResolvedFilter.get(dimension);
+    if (null == currentVals) {
+      currentVals = new ArrayList<DimColumnFilterInfo>(20);
+      currentVals.add(filterResolvedObj);
+      dimensionResolvedFilter.put(dimension, currentVals);
+    } else {
+      currentVals.add(filterResolvedObj);
+    }
+  }
+
+  public Map<CarbonDimension, List<DimColumnFilterInfo>> getDimensionResolvedFilterInstance() {
+    return dimensionResolvedFilter;
+  }
+
+  public Map<Integer, GenericQueryType> getComplexTypesWithBlockStartIndex() {
+    return complexTypesWithBlockStartIndex;
+  }
+
+  public void setComplexTypesWithBlockStartIndex(
+      Map<Integer, GenericQueryType> complexTypesWithBlockStartIndex) {
+    this.complexTypesWithBlockStartIndex = complexTypesWithBlockStartIndex;
+  }
+
+  public CarbonDimension getDimension() {
+    return dimension;
+  }
+
+  public void setDimension(CarbonDimension dimension) {
+    this.dimension = dimension;
+  }
+
+  public int getColumnIndex() {
+    return columnIndex;
+  }
+
+  public void setColumnIndex(int columnIndex) {
+    this.columnIndex = columnIndex;
+  }
+
+  public boolean isNeedCompressedData() {
+    return needCompressedData;
+  }
+
+  public void setNeedCompressedData(boolean needCompressedData) {
+    this.needCompressedData = needCompressedData;
+  }
+
+  public DimColumnFilterInfo getFilterValues() {
+    return resolvedFilterValueObj;
+  }
+
+  public void setFilterValues(final DimColumnFilterInfo resolvedFilterValueObj) {
+    this.resolvedFilterValueObj = resolvedFilterValueObj;
+  }
+
+  public int getRowIndex() {
+    return rowIndex;
+  }
+
+  public void setRowIndex(int rowIndex) {
+    this.rowIndex = rowIndex;
+  }
+
+  public boolean isDimensionExistsInCurrentSilce() {
+    return isDimensionExistsInCurrentSilce;
+  }
+
+  public void setDimensionExistsInCurrentSilce(boolean isDimensionExistsInCurrentSilce) {
+    this.isDimensionExistsInCurrentSilce = isDimensionExistsInCurrentSilce;
+  }
+
+  public int getRsSurrogates() {
+    return rsSurrogates;
+  }
+
+  public void setRsSurrogates(int rsSurrogates) {
+    this.rsSurrogates = rsSurrogates;
+  }
+
+  public String getDefaultValue() {
+    return defaultValue;
+  }
+
+  public void setDefaultValue(String defaultValue) {
+    this.defaultValue = defaultValue;
+  }
+
+  @Override public void populateFilterInfoBasedOnColumnType(ResolvedFilterInfoVisitorIntf visitor,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    if (null != visitor) {
+      visitor.populateFilterResolvedInfo(this, metadata);
+      this.addDimensionResolvedFilterInstance(metadata.getColumnExpression().getDimension(),
+          this.getFilterValues());
+      this.setDimension(metadata.getColumnExpression().getDimension());
+      this.setColumnIndex(metadata.getColumnExpression().getDimension().getOrdinal());
+    }
+
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.java
new file mode 100644
index 0000000..d4cac8c
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/MeasureColumnResolvedFilterInfo.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.carbondata.scan.filter.resolver.resolverinfo;
+
+import java.io.Serializable;
+
+public class MeasureColumnResolvedFilterInfo implements Serializable {
+  /**
+   *
+   */
+  private static final long serialVersionUID = 4222568289115151561L;
+
+  private int columnIndex = -1;
+
+  private int rowIndex = -1;
+
+  private Object uniqueValue;
+
+  private String aggregator;
+
+  private boolean isMeasureExistsInCurrentSlice = true;
+
+  private Object defaultValue;
+
+  private org.carbondata.core.carbon.metadata.datatype.DataType type;
+
+  public int getColumnIndex() {
+    return columnIndex;
+  }
+
+  public void setColumnIndex(int columnIndex) {
+    this.columnIndex = columnIndex;
+  }
+
+  public int getRowIndex() {
+    return rowIndex;
+  }
+
+  public void setRowIndex(int rowIndex) {
+    this.rowIndex = rowIndex;
+  }
+
+  public Object getUniqueValue() {
+    return uniqueValue;
+  }
+
+  public void setUniqueValue(Object uniqueValue) {
+    this.uniqueValue = uniqueValue;
+  }
+
+  public org.carbondata.core.carbon.metadata.datatype.DataType getType() {
+    return type;
+  }
+
+  public void setType(org.carbondata.core.carbon.metadata.datatype.DataType dataType) {
+    this.type = dataType;
+  }
+
+  /**
+   * @return Returns the aggregator.
+   */
+  public String getAggregator() {
+    return aggregator;
+  }
+
+  /**
+   * @param aggregator The aggregator to set.
+   */
+  public void setAggregator(String aggregator) {
+    this.aggregator = aggregator;
+  }
+
+  public boolean isMeasureExistsInCurrentSlice() {
+    return isMeasureExistsInCurrentSlice;
+  }
+
+  public void setMeasureExistsInCurrentSlice(boolean isMeasureExistsInCurrentSlice) {
+    this.isMeasureExistsInCurrentSlice = isMeasureExistsInCurrentSlice;
+  }
+
+  public Object getDefaultValue() {
+    return defaultValue;
+  }
+
+  public void setDefaultValue(double defaultValue) {
+    this.defaultValue = defaultValue;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.java
new file mode 100644
index 0000000..24762ae
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitable/ResolvedFilterInfoVisitable.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.carbondata.scan.filter.resolver.resolverinfo.visitable;
+
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.visitor.ResolvedFilterInfoVisitorIntf;
+
+public interface ResolvedFilterInfoVisitable {
+  /**
+   * This visitable method will visit through the visitor classes which is passed as parameter
+   * and based on different visitor the visitable filter instance will be resolved.
+   *
+   * @param visitor
+   * @param metadata
+   * @throws QueryExecutionException
+   * @throws FilterUnsupportedException
+   */
+  void populateFilterInfoBasedOnColumnType(ResolvedFilterInfoVisitorIntf visitor,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
new file mode 100644
index 0000000..273bdf1
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
@@ -0,0 +1,80 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class CustomTypeDictionaryVisitor implements ResolvedFilterInfoVisitorIntf {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(CustomTypeDictionaryVisitor.class.getName());
+
+  /**
+   * This Visitor method is been used to resolve or populate the filter details
+   * by using custom type dictionary value, the filter membrers will be resolved using
+   * custom type function which will generate dictionary for the direct column type filter members
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws FilterUnsupportedException
+   */
+  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    DimColumnFilterInfo resolvedFilterObject = null;
+
+    List<String> evaluateResultListFinal =
+        metadata.getExpression().evaluate(null).getListAsString();
+    resolvedFilterObject = getDirectDictionaryValKeyMemberForFilter(metadata.getTableIdentifier(),
+        metadata.getColumnExpression(), evaluateResultListFinal, metadata.isIncludeFilter());
+    visitableObj.setFilterValues(resolvedFilterObject);
+  }
+
+  private DimColumnFilterInfo getDirectDictionaryValKeyMemberForFilter(
+      AbsoluteTableIdentifier tableIdentifier, ColumnExpression columnExpression,
+      List<String> evaluateResultListFinal, boolean isIncludeFilter) {
+    List<Integer> surrogates = new ArrayList<Integer>(20);
+    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+        .getDirectDictionaryGenerator(columnExpression.getDimension().getDataType());
+    // Reading the dictionary value direct
+    for (String filterMember : evaluateResultListFinal) {
+      surrogates.add(directDictionaryGenerator.generateDirectSurrogateKey(filterMember));
+    }
+    Collections.sort(surrogates);
+    DimColumnFilterInfo columnFilterInfo = null;
+    if (surrogates.size() > 0) {
+      columnFilterInfo = new DimColumnFilterInfo();
+      columnFilterInfo.setIncludeFilter(isIncludeFilter);
+      columnFilterInfo.setFilterList(surrogates);
+    }
+    return columnFilterInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
new file mode 100644
index 0000000..7ec2751
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.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.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class DictionaryColumnVisitor implements ResolvedFilterInfoVisitorIntf {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DictionaryColumnVisitor.class.getName());
+
+  /**
+   * This Visitor method is used to populate the visitableObj with direct dictionary filter details
+   * where the filters values will be resolve using dictionary cache.
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws QueryExecutionException
+   */
+  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    DimColumnFilterInfo resolvedFilterObject = null;
+    List<String> evaluateResultListFinal =
+        metadata.getExpression().evaluate(null).getListAsString();
+    try {
+      resolvedFilterObject = FilterUtil
+          .getFilterValues(metadata.getTableIdentifier(), metadata.getColumnExpression(),
+              evaluateResultListFinal, metadata.isIncludeFilter());
+    } catch (QueryExecutionException e) {
+      throw new FilterUnsupportedException(e);
+    }
+    visitableObj.setFilterValues(resolvedFilterObject);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
new file mode 100644
index 0000000..35d9d70
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/FilterInfoTypeVisitorFactory.java
@@ -0,0 +1,45 @@
+/*
+ * 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.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.scan.expression.ColumnExpression;
+
+public class FilterInfoTypeVisitorFactory {
+
+  /**
+   * This factory method will be used in order to get the visitor instance based on the
+   * column expression metadata where filters has been applied.
+   *
+   * @param columnExpression
+   * @return
+   */
+  public static ResolvedFilterInfoVisitorIntf getResolvedFilterInfoVisitor(
+      ColumnExpression columnExpression) {
+    if (columnExpression.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+      return new CustomTypeDictionaryVisitor();
+    } else if (!columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return new NoDictionaryTypeVisitor();
+    } else if (columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return new DictionaryColumnVisitor();
+    }
+
+    return null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
new file mode 100644
index 0000000..ee3d157
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.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.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.DimColumnFilterInfo;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public class NoDictionaryTypeVisitor implements ResolvedFilterInfoVisitorIntf {
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(NoDictionaryTypeVisitor.class.getName());
+
+  /**
+   * Visitor Method will update the filter related details in visitableObj, For no dictionary
+   * type columns the filter members will resolved directly, no need to look up in dictionary
+   * since it will not be part of dictionary, directly the actual data can be converted as
+   * byte[] and can be set. this type of encoding is effective when the particular column
+   * is having very high cardinality.
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws FilterUnsupportedException
+   */
+  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException {
+    DimColumnFilterInfo resolvedFilterObject = null;
+    List<String> evaluateResultListFinal =
+        metadata.getExpression().evaluate(null).getListAsString();
+    resolvedFilterObject = FilterUtil
+        .getNoDictionaryValKeyMemberForFilter(metadata.getTableIdentifier(),
+            metadata.getColumnExpression(), evaluateResultListFinal, metadata.isIncludeFilter());
+    visitableObj.setFilterValues(resolvedFilterObject);
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
new file mode 100644
index 0000000..1cd9197
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/ResolvedFilterInfoVisitorIntf.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.filter.resolver.resolverinfo.visitor;
+
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
+import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
+
+public interface ResolvedFilterInfoVisitorIntf {
+
+  /**
+   * Visitor pattern is been used in this scenario inorder to populate the
+   * dimColResolvedFilterInfo visitable object with filter member values based
+   * on the visitor type, currently there 3 types of visitors custom,direct
+   * and no dictionary, all types of visitor populate the visitable instance
+   * as per its buisness logic which is different for all the visitors.
+   *
+   * @param visitableObj
+   * @param metadata
+   * @throws QueryExecutionException
+   */
+  void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
+      FilterResolverMetadata metadata) throws FilterUnsupportedException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
new file mode 100644
index 0000000..840aa02
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
@@ -0,0 +1,243 @@
+/*
+ * 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.carbondata.scan.model;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.scan.expression.Expression;
+
+/**
+ * This class contains all the logical information about the query like dimensions,measures,
+ * sort order, topN etc..
+ */
+public class CarbonQueryPlan implements Serializable {
+  /**
+   *
+   */
+  private static final long serialVersionUID = -9036044826928017164L;
+
+  /**
+   * Schema name , if user asks select * from datasight.employee.
+   * then datasight is the schame name.
+   * Remains null if the user does not select schema name.
+   */
+  private String schemaName;
+
+  /**
+   * Cube name .
+   * if user asks select * from datasight.employee. then employee is the cube name.
+   * It is mandatory.
+   */
+  private String cubeName;
+
+  /**
+   * List of dimensions.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then employee_name
+   * and department_name are dimensions
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<QueryDimension> dimensions =
+      new ArrayList<QueryDimension>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * List of measures.
+   * Ex : select employee_name,department_name,sum(salary) from employee, then sum(salary)
+   * would be measure.
+   * If there is no dimensions asked in query then it would be remained as empty.
+   */
+  private List<QueryMeasure> measures =
+      new ArrayList<QueryMeasure>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  /**
+   * Limit
+   */
+  private int limit = -1;
+
+  /**
+   * If it is detail query, no need to aggregate in backend
+   */
+  private boolean detailQuery;
+
+  /**
+   * expression
+   */
+  private Expression expression;
+
+  /**
+   * queryId
+   */
+  private String queryId;
+
+  /**
+   * outLocationPath
+   */
+  private String outLocationPath;
+
+  /**
+   * isCountStarQuery
+   */
+  private boolean isCountStartQuery;
+
+  private List<QueryDimension> sortedDimensions;
+
+  /**
+   * If it is raw detail query, no need to aggregate in backend. And it reurns with dictionary data
+   * with out decoding.
+   */
+  private boolean rawDetailQuery;
+
+  /**
+   * Constructor created with cube name.
+   *
+   * @param cubeName
+   */
+  public CarbonQueryPlan(String cubeName) {
+    this.cubeName = cubeName;
+  }
+
+  /**
+   * Constructor created with schema name and cube name.
+   *
+   * @param schemaName
+   * @param cubeName
+   */
+  public CarbonQueryPlan(String schemaName, String cubeName) {
+    this.cubeName = cubeName;
+    this.schemaName = schemaName;
+  }
+
+  /**
+   * @return the dimensions
+   */
+  public List<QueryDimension> getDimensions() {
+    return dimensions;
+  }
+
+  public void addDimension(QueryDimension dimension) {
+    this.dimensions.add(dimension);
+  }
+
+  /**
+   * @return the measures
+   */
+  public List<QueryMeasure> getMeasures() {
+    return measures;
+  }
+
+  public void addMeasure(QueryMeasure measure) {
+    this.measures.add(measure);
+  }
+
+  public Expression getFilterExpression() {
+    return expression;
+  }
+
+  public void setFilterExpression(Expression expression) {
+    this.expression = expression;
+  }
+
+  /**
+   * @return the schemaName
+   */
+  public String getSchemaName() {
+    return schemaName;
+  }
+
+  /**
+   * @return the cubeName
+   */
+  public String getCubeName() {
+    return cubeName;
+  }
+
+  /**
+   * @return the limit
+   */
+  public int getLimit() {
+    return limit;
+  }
+
+  /**
+   * @param limit the limit to set
+   */
+  public void setLimit(int limit) {
+    this.limit = limit;
+  }
+
+  /**
+   * @return the detailQuery
+   */
+  public boolean isDetailQuery() {
+    return detailQuery;
+  }
+
+  /**
+   * @param detailQuery the detailQuery to set
+   */
+  public void setDetailQuery(boolean detailQuery) {
+    this.detailQuery = detailQuery;
+  }
+
+  public String getQueryId() {
+    return queryId;
+  }
+
+  public void setQueryId(String queryId) {
+    this.queryId = queryId;
+  }
+
+  public String getOutLocationPath() {
+    return outLocationPath;
+  }
+
+  public void setOutLocationPath(String outLocationPath) {
+    this.outLocationPath = outLocationPath;
+  }
+
+  public boolean isCountStarQuery() {
+    return isCountStartQuery;
+  }
+
+  public void setCountStartQuery(boolean isCountStartQuery) {
+    this.isCountStartQuery = isCountStartQuery;
+  }
+
+  public List<QueryDimension> getSortedDimemsions() {
+    return sortedDimensions;
+  }
+
+  public void setSortedDimemsions(List<QueryDimension> dims) {
+    this.sortedDimensions = dims;
+  }
+
+  public boolean isRawDetailQuery() {
+    return rawDetailQuery;
+  }
+
+  public void setRawDetailQuery(boolean rawDetailQuery) {
+    this.rawDetailQuery = rawDetailQuery;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryColumn.java b/core/src/main/java/org/carbondata/scan/model/QueryColumn.java
new file mode 100644
index 0000000..85ff41d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryColumn.java
@@ -0,0 +1,109 @@
+/*
+ * 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.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * query column  which will have information about column
+ */
+public class QueryColumn implements Serializable {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -4222306600480181084L;
+
+  /**
+   * name of the column
+   */
+  protected String columnName;
+
+  /**
+   * sort order in which column output will be sorted default it will be none
+   */
+  private SortOrderType sortOrder = SortOrderType.NONE;
+
+  /**
+   * query order in which result of the query will be send
+   */
+  private int queryOrder;
+
+  /**
+   * aggregation function applied on column
+   */
+
+  private String aggregationFunction=CarbonCommonConstants.DUMMY;
+
+  public QueryColumn(String columnName) {
+    this.columnName = columnName;
+  }
+
+  /**
+   * @return the sortOrder
+   */
+  public SortOrderType getSortOrder() {
+    return sortOrder;
+  }
+
+  /**
+   * @param sortOrder the sortOrder to set
+   */
+  public void setSortOrder(SortOrderType sortOrder) {
+    this.sortOrder = sortOrder;
+  }
+
+  /**
+   * @return the columnName
+   */
+  public String getColumnName() {
+    return columnName;
+  }
+
+  /**
+   * @return the queryOrder
+   */
+  public int getQueryOrder() {
+    return queryOrder;
+  }
+
+  /**
+   * @param queryOrder the queryOrder to set
+   */
+  public void setQueryOrder(int queryOrder) {
+    this.queryOrder = queryOrder;
+  }
+
+  /**
+   * @return the aggregationFunction
+   */
+  public String getAggregateFunction() {
+    return aggregationFunction;
+  }
+
+  /**
+   * @param aggregationFunction the aggregationFunction to set
+   */
+  public void setAggregateFunction(String aggregationFunction) {
+    this.aggregationFunction = aggregationFunction;
+  }
+
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryDimension.java b/core/src/main/java/org/carbondata/scan/model/QueryDimension.java
new file mode 100644
index 0000000..dc07173
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryDimension.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.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+
+/**
+ * query plan dimension which will holds the information about the query plan dimension
+ * this is done to avoid heavy object serialization
+ */
+public class QueryDimension extends QueryColumn implements Serializable {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = -8492704093776645651L;
+  /**
+   * actual dimension column
+   */
+  private transient CarbonDimension dimension;
+
+  public QueryDimension(String columName) {
+    super(columName);
+  }
+
+  /**
+   * @return the dimension
+   */
+  public CarbonDimension getDimension() {
+    return dimension;
+  }
+
+  /**
+   * @param dimension the dimension to set
+   */
+  public void setDimension(CarbonDimension dimension) {
+    this.dimension = dimension;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java b/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java
new file mode 100644
index 0000000..4035e61
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryMeasure.java
@@ -0,0 +1,61 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+
+/**
+ * query plan measure, this class will holds the information
+ * about measure present in the query, this is done to avoid the serialization
+ * of the heavy object
+ */
+public class QueryMeasure extends QueryColumn implements Serializable {
+
+  /**
+   * serialVersionUID
+   */
+  private static final long serialVersionUID = 1035512411375495414L;
+
+  /**
+   * actual carbon measure object
+   */
+  private transient CarbonMeasure measure;
+
+  public QueryMeasure(String columName) {
+    super(columName);
+  }
+
+  /**
+   * @return the measure
+   */
+  public CarbonMeasure getMeasure() {
+    return measure;
+  }
+
+  /**
+   * @param measure the measure to set
+   */
+  public void setMeasure(CarbonMeasure measure) {
+    this.measure = measure;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QueryModel.java b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
new file mode 100644
index 0000000..c979b6d
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QueryModel.java
@@ -0,0 +1,516 @@
+/*
+ * 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.carbondata.scan.model;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+
+import org.carbondata.core.cache.dictionary.Dictionary;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
+import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.UnknownExpression;
+import org.carbondata.scan.expression.conditional.ConditionalExpression;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+
+/**
+ * Query model which will have all the detail
+ * about the query, This will be sent from driver to executor '
+ * This will be refereed to executing the query.
+ */
+public class QueryModel implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = -4674677234007089052L;
+
+  /**
+   * list of dimension selected for in query
+   */
+  private List<QueryDimension> queryDimension;
+
+  /**
+   * list of dimension in which sorting is applied
+   */
+  private List<QueryDimension> sortDimension;
+
+  /**
+   * list of measure selected in query
+   */
+  private List<QueryMeasure> queryMeasures;
+
+  /**
+   * query id
+   */
+  private String queryId;
+
+  /**
+   * to check if it a aggregate table
+   */
+  private boolean isAggTable;
+
+  /**
+   * filter tree
+   */
+  private FilterResolverIntf filterExpressionResolverTree;
+
+  /**
+   * in case of lime query we need to know how many
+   * records will passed from executor
+   */
+  private int limit;
+
+  /**
+   * to check if it is a count star query , so processing will be different
+   */
+  private boolean isCountStarQuery;
+
+  /**
+   * to check whether aggregation is required during query execution
+   */
+  private boolean detailQuery;
+
+  /**
+   * table block information in which query will be executed
+   */
+  private List<TableBlockInfo> tableBlockInfos;
+
+  /**
+   * sort in which dimension will be get sorted
+   */
+  private byte[] sortOrder;
+
+  /**
+   * absolute table identifier
+   */
+  private AbsoluteTableIdentifier absoluteTableIdentifier;
+  /**
+   * in case of detail query with sort we are spilling to disk
+   * to this location will be used to write the temp file in this location
+   */
+  private String queryTempLocation;
+
+  /**
+   * To handle most of the computation in query engines like spark and hive, carbon should give
+   * raw detailed records to it.
+   */
+  private boolean forcedDetailRawQuery;
+
+  /**
+   * paritition column list
+   */
+  private List<String> paritionColumns;
+
+  /**
+   * this will hold the information about the dictionary dimension
+   * which to
+   */
+  public transient Map<String, Dictionary> columnToDictionaryMapping;
+
+  /**
+   * table on which query will be executed
+   * TODO need to remove this ad pass only the path
+   * and carbon metadata will load the table from metadata file
+   */
+  private CarbonTable table;
+
+  /**
+   * This is used only whne [forcedDetailRawQuery = true]. By default forcedDetailRawQuery returns
+   * dictionary values. But user wants in detail raw bytes the user set this field to true.
+   */
+  private boolean rawBytesDetailQuery;
+
+  public QueryModel() {
+    tableBlockInfos = new ArrayList<TableBlockInfo>();
+    queryDimension = new ArrayList<QueryDimension>();
+    queryMeasures = new ArrayList<QueryMeasure>();
+    sortDimension = new ArrayList<QueryDimension>();
+    sortOrder = new byte[0];
+    paritionColumns = new ArrayList<String>();
+
+  }
+
+  public static QueryModel createModel(AbsoluteTableIdentifier absoluteTableIdentifier,
+      CarbonQueryPlan queryPlan, CarbonTable carbonTable) {
+    QueryModel queryModel = new QueryModel();
+    String factTableName = carbonTable.getFactTableName();
+    queryModel.setAbsoluteTableIdentifier(absoluteTableIdentifier);
+
+    fillQueryModel(queryPlan, carbonTable, queryModel, factTableName);
+
+    queryModel.setLimit(queryPlan.getLimit());
+    queryModel.setDetailQuery(queryPlan.isDetailQuery());
+    queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery());
+    queryModel.setQueryId(queryPlan.getQueryId());
+    queryModel.setQueryTempLocation(queryPlan.getOutLocationPath());
+    return queryModel;
+  }
+
+  private static void fillQueryModel(CarbonQueryPlan queryPlan, CarbonTable carbonTable,
+      QueryModel queryModel, String factTableName) {
+    queryModel.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
+    queryModel.setQueryDimension(queryPlan.getDimensions());
+    fillSortInfoInModel(queryModel, queryPlan.getSortedDimemsions());
+    queryModel.setQueryMeasures(
+        queryPlan.getMeasures());
+    if (null != queryPlan.getFilterExpression()) {
+      processFilterExpression(queryPlan.getFilterExpression(),
+          carbonTable.getDimensionByTableName(factTableName),
+          carbonTable.getMeasureByTableName(factTableName));
+    }
+    queryModel.setCountStarQuery(queryPlan.isCountStarQuery());
+    //TODO need to remove this code, and executor will load the table
+    // from file metadata
+    queryModel.setTable(carbonTable);
+  }
+
+  private static void fillSortInfoInModel(QueryModel executorModel,
+      List<QueryDimension> sortedDims) {
+    if (null != sortedDims) {
+      byte[] sortOrderByteArray = new byte[sortedDims.size()];
+      int i = 0;
+      for (QueryColumn mdim : sortedDims) {
+        sortOrderByteArray[i++] = (byte) mdim.getSortOrder().ordinal();
+      }
+      executorModel.setSortOrder(sortOrderByteArray);
+      executorModel.setSortDimension(sortedDims);
+    } else {
+      executorModel.setSortOrder(new byte[0]);
+      executorModel.setSortDimension(new ArrayList<QueryDimension>(0));
+    }
+
+  }
+
+  public static void processFilterExpression(
+      Expression filterExpression, List<CarbonDimension> dimensions, List<CarbonMeasure> measures) {
+    if (null != filterExpression) {
+      if (null != filterExpression.getChildren() && filterExpression.getChildren().size() == 0) {
+        if (filterExpression instanceof ConditionalExpression) {
+          List<ColumnExpression> listOfCol =
+              ((ConditionalExpression) filterExpression).getColumnList();
+          for (ColumnExpression expression : listOfCol) {
+            setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression);
+          }
+
+        }
+      }
+      for (Expression expression : filterExpression.getChildren()) {
+
+        if (expression instanceof ColumnExpression) {
+          setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression);
+        } else if (expression instanceof UnknownExpression) {
+          UnknownExpression exp = ((UnknownExpression) expression);
+          List<ColumnExpression> listOfColExpression = exp.getAllColumnList();
+          for (ColumnExpression col : listOfColExpression) {
+            setDimAndMsrColumnNode(dimensions, measures, col);
+          }
+        } else {
+          processFilterExpression(expression, dimensions, measures);
+        }
+      }
+    }
+
+  }
+
+  private static CarbonMeasure getCarbonMetadataMeasure(String name, List<CarbonMeasure> measures) {
+    for (CarbonMeasure measure : measures) {
+      if (measure.getColName().equalsIgnoreCase(name)) {
+        return measure;
+      }
+    }
+    return null;
+  }
+
+  private static void setDimAndMsrColumnNode(List<CarbonDimension> dimensions,
+      List<CarbonMeasure> measures, ColumnExpression col) {
+    CarbonDimension dim;
+    CarbonMeasure msr;
+    String columnName;
+    columnName = col.getColumnName();
+    dim = CarbonUtil.findDimension(dimensions, columnName);
+    col.setCarbonColumn(dim);
+    col.setDimension(dim);
+    col.setDimension(true);
+    if (null == dim) {
+      msr = getCarbonMetadataMeasure(columnName, measures);
+      col.setCarbonColumn(msr);
+      col.setDimension(false);
+    }
+  }
+
+  /**
+   * It gets the projection columns
+   */
+  public CarbonColumn[] getProjectionColumns() {
+    CarbonColumn[] carbonColumns =
+        new CarbonColumn[getQueryDimension().size() + getQueryMeasures()
+            .size()];
+    for (QueryDimension dimension : getQueryDimension()) {
+      carbonColumns[dimension.getQueryOrder()] = dimension.getDimension();
+    }
+    for (QueryMeasure msr : getQueryMeasures()) {
+      carbonColumns[msr.getQueryOrder()] = msr.getMeasure();
+    }
+    return carbonColumns;
+  }
+
+  /**
+   * @return the queryDimension
+   */
+  public List<QueryDimension> getQueryDimension() {
+    return queryDimension;
+  }
+
+  /**
+   * @param queryDimension the queryDimension to set
+   */
+  public void setQueryDimension(List<QueryDimension> queryDimension) {
+    this.queryDimension = queryDimension;
+  }
+
+  /**
+   * @return the queryMeasures
+   */
+  public List<QueryMeasure> getQueryMeasures() {
+    return queryMeasures;
+  }
+
+  /**
+   * @param queryMeasures the queryMeasures to set
+   */
+  public void setQueryMeasures(List<QueryMeasure> queryMeasures) {
+    this.queryMeasures = queryMeasures;
+  }
+
+  /**
+   * @return the queryId
+   */
+  public String getQueryId() {
+    return queryId;
+  }
+
+  /**
+   * @param queryId the queryId to set
+   */
+  public void setQueryId(String queryId) {
+    this.queryId = queryId;
+  }
+
+  /**
+   * @return the isAggTable
+   */
+  public boolean isAggTable() {
+    return isAggTable;
+  }
+
+  /**
+   * @param isAggTable the isAggTable to set
+   */
+  public void setAggTable(boolean isAggTable) {
+    this.isAggTable = isAggTable;
+  }
+
+  /**
+   * @return the limit
+   */
+  public int getLimit() {
+    return limit;
+  }
+
+  /**
+   * @param limit the limit to set
+   */
+  public void setLimit(int limit) {
+    this.limit = limit;
+  }
+
+  /**
+   * @return the isCountStarQuery
+   */
+  public boolean isCountStarQuery() {
+    return isCountStarQuery;
+  }
+
+  /**
+   * @param isCountStarQuery the isCountStarQuery to set
+   */
+  public void setCountStarQuery(boolean isCountStarQuery) {
+    this.isCountStarQuery = isCountStarQuery;
+  }
+
+  /**
+   * @return the isdetailQuery
+   */
+  public boolean isDetailQuery() {
+    return detailQuery;
+  }
+
+  public void setDetailQuery(boolean detailQuery) {
+    this.detailQuery = detailQuery;
+  }
+
+  /**
+   * @return the tableBlockInfos
+   */
+  public List<TableBlockInfo> getTableBlockInfos() {
+    return tableBlockInfos;
+  }
+
+  /**
+   * @param tableBlockInfos the tableBlockInfos to set
+   */
+  public void setTableBlockInfos(List<TableBlockInfo> tableBlockInfos) {
+    this.tableBlockInfos = tableBlockInfos;
+  }
+
+  /**
+   * @return the queryTempLocation
+   */
+  public String getQueryTempLocation() {
+    return queryTempLocation;
+  }
+
+  /**
+   * @param queryTempLocation the queryTempLocation to set
+   */
+  public void setQueryTempLocation(String queryTempLocation) {
+    this.queryTempLocation = queryTempLocation;
+  }
+
+  /**
+   * @return the sortOrder
+   */
+  public byte[] getSortOrder() {
+    return sortOrder;
+  }
+
+  /**
+   * @param sortOrder the sortOrder to set
+   */
+  public void setSortOrder(byte[] sortOrder) {
+    this.sortOrder = sortOrder;
+  }
+
+  /**
+   * @return the sortDimension
+   */
+  public List<QueryDimension> getSortDimension() {
+    return sortDimension;
+  }
+
+  /**
+   * @param sortDimension the sortDimension to set
+   */
+  public void setSortDimension(List<QueryDimension> sortDimension) {
+    this.sortDimension = sortDimension;
+  }
+
+  /**
+   * @return the filterEvaluatorTree
+   */
+  public FilterResolverIntf getFilterExpressionResolverTree() {
+    return filterExpressionResolverTree;
+  }
+
+  public void setFilterExpressionResolverTree(FilterResolverIntf filterExpressionResolverTree) {
+    this.filterExpressionResolverTree = filterExpressionResolverTree;
+  }
+
+  /**
+   * @return the absoluteTableIdentifier
+   */
+  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
+    return absoluteTableIdentifier;
+  }
+
+  /**
+   * @param absoluteTableIdentifier the absoluteTableIdentifier to set
+   */
+  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
+    this.absoluteTableIdentifier = absoluteTableIdentifier;
+  }
+
+  /**
+   * @return the paritionColumns
+   */
+  public List<String> getParitionColumns() {
+    return paritionColumns;
+  }
+
+  /**
+   * @param paritionColumns the paritionColumns to set
+   */
+  public void setParitionColumns(List<String> paritionColumns) {
+    this.paritionColumns = paritionColumns;
+  }
+
+  /**
+   * @return the table
+   */
+  public CarbonTable getTable() {
+    return table;
+  }
+
+  /**
+   * @param table the table to set
+   */
+  public void setTable(CarbonTable table) {
+    this.table = table;
+  }
+
+  public boolean isForcedDetailRawQuery() {
+    return forcedDetailRawQuery;
+  }
+
+  public void setForcedDetailRawQuery(boolean forcedDetailRawQuery) {
+    this.forcedDetailRawQuery = forcedDetailRawQuery;
+  }
+
+  /**
+   * @return
+   */
+  public Map<String, Dictionary> getColumnToDictionaryMapping() {
+    return columnToDictionaryMapping;
+  }
+
+  /**
+   * @param columnToDictionaryMapping
+   */
+  public void setColumnToDictionaryMapping(Map<String, Dictionary> columnToDictionaryMapping) {
+    this.columnToDictionaryMapping = columnToDictionaryMapping;
+  }
+
+  public boolean isRawBytesDetailQuery() {
+    return rawBytesDetailQuery;
+  }
+
+  public void setRawBytesDetailQuery(boolean rawBytesDetailQuery) {
+    this.rawBytesDetailQuery = rawBytesDetailQuery;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.java b/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.java
new file mode 100644
index 0000000..643e81b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/QuerySchemaInfo.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.carbondata.scan.model;
+
+import java.io.Serializable;
+
+import org.carbondata.core.keygenerator.KeyGenerator;
+
+public class QuerySchemaInfo implements Serializable {
+
+  private int[] maskedByteIndexes;
+
+  private KeyGenerator keyGenerator;
+
+  private QueryDimension[] queryDimensions;
+
+  private QueryMeasure[] queryMeasures;
+
+  private int[] queryOrder;
+
+  private int[] queryReverseOrder;
+
+  public int[] getMaskedByteIndexes() {
+    return maskedByteIndexes;
+  }
+
+  public void setMaskedByteIndexes(int[] maskedByteIndexes) {
+    this.maskedByteIndexes = maskedByteIndexes;
+  }
+
+  public KeyGenerator getKeyGenerator() {
+    return keyGenerator;
+  }
+
+  public void setKeyGenerator(KeyGenerator keyGenerator) {
+    this.keyGenerator = keyGenerator;
+  }
+
+  public QueryDimension[] getQueryDimensions() {
+    return queryDimensions;
+  }
+
+  public void setQueryDimensions(QueryDimension[] queryDimensions) {
+    this.queryDimensions = queryDimensions;
+  }
+
+  public QueryMeasure[] getQueryMeasures() {
+    return queryMeasures;
+  }
+
+  public void setQueryMeasures(QueryMeasure[] queryMeasures) {
+    this.queryMeasures = queryMeasures;
+  }
+
+  public int[] getQueryOrder() {
+    return queryOrder;
+  }
+
+  public void setQueryOrder(int[] queryOrder) {
+    this.queryOrder = queryOrder;
+  }
+
+  public int[] getQueryReverseOrder() {
+    return queryReverseOrder;
+  }
+
+  public void setQueryReverseOrder(int[] queryReverseOrder) {
+    this.queryReverseOrder = queryReverseOrder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/model/SortOrderType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/SortOrderType.java b/core/src/main/java/org/carbondata/scan/model/SortOrderType.java
new file mode 100644
index 0000000..00b9219
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/model/SortOrderType.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.carbondata.scan.model;
+
+/**
+ * enum for sorting the columns
+ */
+public enum SortOrderType {
+
+    /**
+     * Ascending order
+     */
+    ASC(0),
+
+    /**
+     * Descending order.
+     */
+    DSC(1),
+
+    /**
+     * No order mentioned
+     */
+    NONE(-1);
+  /**
+   * Order type in numeric
+   */
+  private int orderType;
+
+  SortOrderType(int orderType) {
+    this.orderType = orderType;
+  }
+
+  /**
+   * Order type in number
+   *
+   * @return orderType int
+   */
+  public int getOrderType() {
+    return orderType;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
new file mode 100644
index 0000000..e3c7cd5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
@@ -0,0 +1,126 @@
+/*
+ * 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.carbondata.scan.processor;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.scan.collector.ScannedResultCollector;
+import org.carbondata.scan.collector.impl.ListBasedResultCollector;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.BlockExecutionInfo;
+import org.carbondata.scan.result.AbstractScannedResult;
+import org.carbondata.scan.result.Result;
+import org.carbondata.scan.scanner.BlockletScanner;
+import org.carbondata.scan.scanner.impl.FilterScanner;
+import org.carbondata.scan.scanner.impl.NonFilterScanner;
+
+/**
+ * This abstract class provides a skeletal implementation of the
+ * Block iterator.
+ */
+public abstract class AbstractDataBlockIterator extends CarbonIterator<Result> {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName());
+  /**
+   * iterator which will be used to iterate over data blocks
+   */
+  protected CarbonIterator<DataRefNode> dataBlockIterator;
+
+  /**
+   * execution details
+   */
+  protected BlockExecutionInfo blockExecutionInfo;
+
+  /**
+   * result collector which will be used to aggregate the scanned result
+   */
+  protected ScannedResultCollector scannerResultAggregator;
+
+  /**
+   * processor which will be used to process the block processing can be
+   * filter processing or non filter processing
+   */
+  protected BlockletScanner blockletScanner;
+
+  /**
+   * to hold the data block
+   */
+  protected BlocksChunkHolder blocksChunkHolder;
+
+  /**
+   * batch size of result
+   */
+  protected int batchSize;
+
+  protected AbstractScannedResult scannedResult;
+
+  public AbstractDataBlockIterator(BlockExecutionInfo blockExecutionInfo,
+      FileHolder fileReader, int batchSize) {
+    this.blockExecutionInfo = blockExecutionInfo;
+    dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
+        blockExecutionInfo.getNumberOfBlockToScan());
+    blocksChunkHolder = new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(),
+        blockExecutionInfo.getTotalNumberOfMeasureBlock());
+    blocksChunkHolder.setFileReader(fileReader);
+
+    if (blockExecutionInfo.getFilterExecuterTree() != null) {
+      blockletScanner = new FilterScanner(blockExecutionInfo);
+    } else {
+      blockletScanner = new NonFilterScanner(blockExecutionInfo);
+    }
+
+    this.scannerResultAggregator =
+        new ListBasedResultCollector(blockExecutionInfo);
+    this.batchSize = batchSize;
+  }
+
+  public boolean hasNext() {
+    try {
+      if (scannedResult != null && scannedResult.hasNext()) {
+        return true;
+      } else {
+        scannedResult = getNextScannedResult();
+        while (scannedResult != null) {
+          if (scannedResult.hasNext()) {
+            return true;
+          }
+          scannedResult = getNextScannedResult();
+        }
+        return false;
+      }
+    } catch (QueryExecutionException ex) {
+      throw new RuntimeException(ex);
+    }
+  }
+
+  private AbstractScannedResult getNextScannedResult() throws QueryExecutionException {
+    if (dataBlockIterator.hasNext()) {
+      blocksChunkHolder.setDataBlock(dataBlockIterator.next());
+      blocksChunkHolder.reset();
+      return blockletScanner.scanBlocklet(blocksChunkHolder);
+    }
+    return null;
+  }
+
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
new file mode 100644
index 0000000..0b36ab5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.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.carbondata.scan.processor;
+
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.iterator.CarbonIterator;
+
+/**
+ * Below class will be used to iterate over data block
+ */
+public class BlockletIterator extends CarbonIterator<DataRefNode> {
+  /**
+   * data store block
+   */
+  protected DataRefNode datablock;
+  /**
+   * block counter to keep a track how many block has been processed
+   */
+  private int blockCounter;
+
+  /**
+   * flag to be used to check any more data block is present or not
+   */
+  private boolean hasNext = true;
+
+  /**
+   * total number blocks assgned to this iterator
+   */
+  private long totalNumberOfBlocksToScan;
+
+  /**
+   * Constructor
+   *
+   * @param datablock                 first data block
+   * @param totalNumberOfBlocksToScan total number of blocks to be scanned
+   */
+  public BlockletIterator(DataRefNode datablock, long totalNumberOfBlocksToScan) {
+    this.datablock = datablock;
+    this.totalNumberOfBlocksToScan = totalNumberOfBlocksToScan;
+  }
+
+  /**
+   * is all the blocks assigned to this iterator has been processed
+   */
+  @Override public boolean hasNext() {
+    return hasNext;
+  }
+
+  @Override
+  /**
+   * To get the next block
+   * @return next data block
+   *
+   */
+  public DataRefNode next() {
+    // get the current blocks
+    DataRefNode datablockTemp = datablock;
+    // store the next data block
+    datablock = datablock.getNextDataRefNode();
+    // increment the counter
+    blockCounter++;
+    // if all the data block is processed then
+    // set the has next flag to false
+    // or if number of blocks assigned to this iterator is processed
+    // then also set the hasnext flag to false
+    if (null == datablock || blockCounter >= this.totalNumberOfBlocksToScan) {
+      hasNext = false;
+    }
+    return datablockTemp;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java b/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java
new file mode 100644
index 0000000..bbf0cd9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/processor/BlocksChunkHolder.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.processor;
+
+import org.carbondata.core.carbon.datastore.DataRefNode;
+import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
+import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
+import org.carbondata.core.datastorage.store.FileHolder;
+
+/**
+ * Block chunk holder which will hold the dimension and
+ * measure chunk
+ */
+public class BlocksChunkHolder {
+
+  /**
+   * dimension column data chunk
+   */
+  private DimensionColumnDataChunk[] dimensionDataChunk;
+
+  /**
+   * measure column data chunk
+   */
+  private MeasureColumnDataChunk[] measureDataChunk;
+
+  /**
+   * file reader which will use to read the block from file
+   */
+  private FileHolder fileReader;
+
+  /**
+   * data block
+   */
+  private DataRefNode dataBlock;
+
+  public BlocksChunkHolder(int numberOfDimensionBlock, int numberOfMeasureBlock) {
+    dimensionDataChunk = new DimensionColumnDataChunk[numberOfDimensionBlock];
+    measureDataChunk = new MeasureColumnDataChunk[numberOfMeasureBlock];
+  }
+
+  /**
+   * @return the dimensionDataChunk
+   */
+  public DimensionColumnDataChunk[] getDimensionDataChunk() {
+    return dimensionDataChunk;
+  }
+
+  /**
+   * @param dimensionDataChunk the dimensionDataChunk to set
+   */
+  public void setDimensionDataChunk(DimensionColumnDataChunk[] dimensionDataChunk) {
+    this.dimensionDataChunk = dimensionDataChunk;
+  }
+
+  /**
+   * @return the measureDataChunk
+   */
+  public MeasureColumnDataChunk[] getMeasureDataChunk() {
+    return measureDataChunk;
+  }
+
+  /**
+   * @param measureDataChunk the measureDataChunk to set
+   */
+  public void setMeasureDataChunk(MeasureColumnDataChunk[] measureDataChunk) {
+    this.measureDataChunk = measureDataChunk;
+  }
+
+  /**
+   * @return the fileReader
+   */
+  public FileHolder getFileReader() {
+    return fileReader;
+  }
+
+  /**
+   * @param fileReader the fileReader to set
+   */
+  public void setFileReader(FileHolder fileReader) {
+    this.fileReader = fileReader;
+  }
+
+  /**
+   * @return the dataBlock
+   */
+  public DataRefNode getDataBlock() {
+    return dataBlock;
+  }
+
+  /**
+   * @param dataBlock the dataBlock to set
+   */
+  public void setDataBlock(DataRefNode dataBlock) {
+    this.dataBlock = dataBlock;
+  }
+
+  /***
+   * To reset the measure chunk and dimension chunk
+   * array
+   */
+  public void reset() {
+    for (int i = 0; i < measureDataChunk.length; i++) {
+      this.measureDataChunk[i] = null;
+    }
+    for (int i = 0; i < dimensionDataChunk.length; i++) {
+      this.dimensionDataChunk[i] = null;
+    }
+  }
+}



[17/56] [abbrv] incubator-carbondata git commit: Refactored core package and fixed all testcases (#684)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java
deleted file mode 100644
index d3af870..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailWithOrderByQueryExecutor.java
+++ /dev/null
@@ -1,53 +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.carbondata.query.carbon.executor.impl;
-
-import java.util.List;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.executor.internal.impl.InternalDetailWithOrderQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.RowResult;
-import org.carbondata.query.carbon.result.iterator.ChunkBasedResultIterator;
-import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
-
-/**
- * Below method will be used to execute the detail query with order by
- */
-public class DetailWithOrderByQueryExecutor extends AbstractQueryExecutor {
-
-  @Override public CarbonIterator<RowResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    // get the execution info
-    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    // in case of sorting we need to add sort information only for last block as
-    // all the previous block data will be updated based last block and after
-    // processing of all the block sorting will be applied
-    blockExecutionInfoList.get(blockExecutionInfoList.size() - 1)
-        .setSortInfo(getSortInfos(queryModel));
-    InternalQueryExecutor internalQueryExecutor = new InternalDetailWithOrderQueryExecutor();
-    return new ChunkRowIterator(new ChunkBasedResultIterator(
-        internalQueryExecutor.executeQuery(blockExecutionInfoList, null), queryProperties,
-        queryModel));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java
deleted file mode 100644
index 563201d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/FunctionQueryExecutor.java
+++ /dev/null
@@ -1,37 +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.carbondata.query.carbon.executor.impl;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.RowResult;
-
-/**
- * Below class will be used execute the function query
- * for example:count(1)
- */
-public class FunctionQueryExecutor extends CountStarQueryExecutor {
-
-  @Override public CarbonIterator<RowResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    queryProperties.isFunctionQuery = true;
-    return super.execute(queryModel);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
index 1a30120..ff42fbc 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
@@ -23,7 +23,7 @@ import java.util.Map;
 
 import org.carbondata.core.cache.dictionary.Dictionary;
 import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
 
 /**
@@ -66,13 +66,13 @@ public class QueryExecutorProperties {
   public byte[] sortDimIndexes;
 
   /**
-   * aggregator class selected for all aggregation function selected in query
-   */
-  public MeasureAggregator[] measureAggregators;
-
-  /**
    * this will hold the information about the dictionary dimension
    * which to
    */
   public Map<String, Dictionary> columnToDictionayMapping;
+
+  /**
+   * Measure datatypes
+   */
+  public DataType[] measureDataTypes;
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
index fa44437..c9e9732 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/infos/AggregatorInfo.java
@@ -18,7 +18,7 @@
  */
 package org.carbondata.query.carbon.executor.infos;
 
-import org.carbondata.query.aggregator.MeasureAggregator;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
 
 /**
  * Info class which store all the details
@@ -27,13 +27,6 @@ import org.carbondata.query.aggregator.MeasureAggregator;
 public class AggregatorInfo {
 
   /**
-   * measure aggregator array instance which will be used to
-   * aggregate the aggregate function columns
-   * it can be any dimension or measure column
-   */
-  private MeasureAggregator[] measuresAggreagators;
-
-  /**
    * selected query measure ordinal
    * which will be used to read the measures chunk data
    * this will be storing the index of the measure in measures chunk
@@ -72,18 +65,9 @@ public class AggregatorInfo {
   private int measureAggregatorStartIndex;
 
   /**
-   * @return the measuresAggreagators
-   */
-  public MeasureAggregator[] getMeasuresAggreagators() {
-    return measuresAggreagators;
-  }
-
-  /**
-   * @param measuresAggreagators the measuresAggreagators to set
+   * Datatype of each measure;
    */
-  public void setMeasuresAggreagators(MeasureAggregator[] measuresAggreagators) {
-    this.measuresAggreagators = measuresAggreagators;
-  }
+  private DataType[] measureDataTypes;
 
   /**
    * @return the measureOrdinal
@@ -154,4 +138,12 @@ public class AggregatorInfo {
   public void setMeasureAggregatorStartIndex(int measureAggregatorStartIndex) {
     this.measureAggregatorStartIndex = measureAggregatorStartIndex;
   }
+
+  public DataType[] getMeasureDataTypes() {
+    return measureDataTypes;
+  }
+
+  public void setMeasureDataTypes(DataType[] measureDataTypes) {
+    this.measureDataTypes = measureDataTypes;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
index 894be0d..7bed33d 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/infos/BlockExecutionInfo.java
@@ -18,7 +18,6 @@
  */
 package org.carbondata.query.carbon.executor.infos;
 
-import java.util.List;
 import java.util.Map;
 
 import org.carbondata.core.cache.dictionary.Dictionary;
@@ -27,9 +26,7 @@ import org.carbondata.core.carbon.datastore.IndexKey;
 import org.carbondata.core.carbon.datastore.block.AbstractIndex;
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
 import org.carbondata.query.carbon.merger.ScannedResultMerger;
-import org.carbondata.query.carbon.model.CustomAggregateExpression;
 import org.carbondata.query.filter.executer.FilterExecuter;
 
 /**
@@ -92,12 +89,6 @@ public class BlockExecutionInfo {
   private IndexKey endKey;
 
   /**
-   * holder of custom aggregation details which will be used to aggregate the
-   * custome function UDAF
-   */
-  private List<CustomAggregateExpression> customAggregateExpressions;
-
-  /**
    * masked byte for block which will be used to unpack the fixed length key,
    * this will be used for updating the older block key with new block key
    * generator
@@ -173,11 +164,6 @@ public class BlockExecutionInfo {
    * key generator used for generating the table block fixed length key
    */
   private KeyGenerator blockKeyGenerator;
-  /**
-   * dimension aggregator list which will be used to aggregate the dimension
-   * data
-   */
-  private List<DimensionDataAggregator> dimensionAggregator;
 
   /**
    * each column value size
@@ -216,6 +202,11 @@ public class BlockExecutionInfo {
   private boolean isRawRecordDetailQuery;
 
   /**
+   * whether dimensions exist in query.
+   */
+  private boolean isDimensionsExistInQuery;
+
+  /**
    * @return the tableBlock
    */
   public AbstractIndex getDataBlock() {
@@ -328,21 +319,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the customAggregateExpression
-   */
-  public List<CustomAggregateExpression> getCustomAggregateExpressions() {
-    return customAggregateExpressions;
-  }
-
-  /**
-   * @param customAggregateExpression the customAggregateExpression to set
-   */
-  public void setCustomAggregateExpressions(
-      List<CustomAggregateExpression> customAggregateExpressions) {
-    this.customAggregateExpressions = customAggregateExpressions;
-  }
-
-  /**
    * @return the maskedByteForBlock
    */
   public int[] getMaskedByteForBlock() {
@@ -539,20 +515,6 @@ public class BlockExecutionInfo {
   }
 
   /**
-   * @return the dimensionAggregator
-   */
-  public List<DimensionDataAggregator> getDimensionAggregator() {
-    return dimensionAggregator;
-  }
-
-  /**
-   * @param dimensionAggregator the dimensionAggregator to set
-   */
-  public void setDimensionAggregator(List<DimensionDataAggregator> dimensionAggregator) {
-    this.dimensionAggregator = dimensionAggregator;
-  }
-
-  /**
    * @return the eachColumnValueSize
    */
   public int[] getEachColumnValueSize() {
@@ -658,4 +620,12 @@ public class BlockExecutionInfo {
   public void setRawRecordDetailQuery(boolean rawRecordDetailQuery) {
     isRawRecordDetailQuery = rawRecordDetailQuery;
   }
+
+  public boolean isDimensionsExistInQuery() {
+    return isDimensionsExistInQuery;
+  }
+
+  public void setDimensionsExistInQuery(boolean dimensionsExistInQuery) {
+    isDimensionsExistInQuery = dimensionsExistInQuery;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
index 8a6268c..a8c2c3a 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAbstractQueryExecutor.java
@@ -36,7 +36,6 @@ import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
 import org.carbondata.query.carbon.merger.ScannedResultMerger;
-import org.carbondata.query.carbon.merger.impl.SortedScannedResultMerger;
 import org.carbondata.query.carbon.merger.impl.UnSortedScannedResultMerger;
 import org.carbondata.query.carbon.result.Result;
 
@@ -66,9 +65,9 @@ public abstract class InternalAbstractQueryExecutor implements InternalQueryExec
    * Below method will be used to used to execute the detail query
    * and it will return iterator over result
    *
-   * @param executionInfos block execution info which will have all the properties
+   * @param tableBlockExecutionInfosList block execution info which will have all the properties
    *                       required for query execution
-   * @param sliceIndexes   slice indexes to be executed in this case it w
+   * @param sliceIndex   slice indexes to be executed in this case it w
    * @return query result
    */
   @Override public CarbonIterator<Result> executeQuery(
@@ -80,12 +79,7 @@ public abstract class InternalAbstractQueryExecutor implements InternalQueryExec
         tableBlockExecutionInfosList.get(tableBlockExecutionInfosList.size() - 1);
     execService = Executors.newFixedThreadPool(numberOfCores);
     ScannedResultMerger scannedResultProcessor = null;
-    if (null != latestInfo.getSortInfo()
-        && latestInfo.getSortInfo().getSortDimensionIndex().length > 0) {
-      scannedResultProcessor = new SortedScannedResultMerger(latestInfo, numberOfCores);
-    } else {
-      scannedResultProcessor = new UnSortedScannedResultMerger(latestInfo, numberOfCores);
-    }
+    scannedResultProcessor = new UnSortedScannedResultMerger(latestInfo, numberOfCores);
     try {
       List<Future> listFutureObjects = new ArrayList<Future>();
       for (BlockExecutionInfo blockInfo : tableBlockExecutionInfosList) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java
deleted file mode 100644
index 4c2a1a3..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalAggregationQueryExecutor.java
+++ /dev/null
@@ -1,44 +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.carbondata.query.carbon.executor.internal.impl;
-
-import java.util.concurrent.Executors;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-/**
- * Below class will be used to execute the aggregated query
- */
-public class InternalAggregationQueryExecutor extends InternalAbstractQueryExecutor {
-
-  public InternalAggregationQueryExecutor() {
-
-    // get the number of core can be used to execute the query
-    try {
-      numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException e) {
-      numberOfCores = 1;
-    }
-    // create thread pool
-    execService = Executors.newFixedThreadPool(numberOfCores);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
deleted file mode 100644
index 14c336d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalCountStartQueryExecutor.java
+++ /dev/null
@@ -1,83 +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.carbondata.query.carbon.executor.internal.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.aggregator.MeasureAggregator;
-import org.carbondata.query.aggregator.impl.count.CountAggregator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.impl.ListBasedResult;
-import org.carbondata.query.carbon.result.iterator.MemoryBasedResultIterator;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * Below class will be used to execute the count star query or any function query
- * like count(1) , in this case block scanning is not required.
- */
-public class InternalCountStartQueryExecutor implements InternalQueryExecutor {
-
-  /**
-   * data block available for query execution
-   */
-  private List<AbstractIndex> blockList;
-
-  public InternalCountStartQueryExecutor(List<AbstractIndex> blockList) {
-    this.blockList = blockList;
-  }
-
-  /**
-   * Method to execute the count start query
-   *
-   * @param block execution info
-   * @param slice indexes
-   */
-  public CarbonIterator<Result> executeQuery(List<BlockExecutionInfo> infos, int[] sliceIndex)
-      throws QueryExecutionException {
-    long count = 0;
-    // for each block get the total number of rows
-    for (AbstractIndex tableBlock : this.blockList) {
-      count += tableBlock.getTotalNumberOfRows();
-    }
-    // as this is a count start need to create counter star aggregator
-    MeasureAggregator[] countAgg = new MeasureAggregator[1];
-    countAgg[0] = new CountAggregator();
-    countAgg[0].setNewValue(count);
-
-    ListBasedResultWrapper resultWrapper = new ListBasedResultWrapper();
-    Result<List<ListBasedResultWrapper>> result = new ListBasedResult();
-    ByteArrayWrapper wrapper = new ByteArrayWrapper();
-    wrapper.setDictionaryKey(new byte[0]);
-    resultWrapper.setKey(wrapper);
-    resultWrapper.setValue(countAgg);
-    List<ListBasedResultWrapper> wrapperList = new ArrayList<ListBasedResultWrapper>(1);
-    wrapperList.add(resultWrapper);
-    result.addScannedResult(wrapperList);
-    // returning the iterator over the result
-    return new MemoryBasedResultIterator(result);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
index c7bfa6b..866596f 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
@@ -70,7 +70,7 @@ public class InternalDetailQueryExecutor implements InternalQueryExecutor {
         recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
       }
     }
-    this.numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
+    numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
         .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
             CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
     if (numberOfCores == 0) {
@@ -87,10 +87,11 @@ public class InternalDetailQueryExecutor implements InternalQueryExecutor {
    * @param sliceIndexes   slice indexes to be executed
    * @return query result
    */
-  @Override public CarbonIterator<Result> executeQuery(List<BlockExecutionInfo> executionInfos,
+  @Override public CarbonIterator<Result> executeQuery(
+      List<BlockExecutionInfo> executionInfos,
       int[] sliceIndexes) throws QueryExecutionException {
     long startTime = System.currentTimeMillis();
-    QueryRunner task = null;
+    QueryRunner task;
     ScannedResultMerger scannedResultProcessor =
         new UnSortedScannedResultMerger(executionInfos.get(executionInfos.size() - 1),
             sliceIndexes.length);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java
deleted file mode 100644
index c20a265..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailWithOrderQueryExecutor.java
+++ /dev/null
@@ -1,74 +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.carbondata.query.carbon.executor.internal.impl;
-
-import java.util.concurrent.Executors;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-/**
- * Below class will be used to execute the detail query with order by
- */
-public class InternalDetailWithOrderQueryExecutor extends InternalAbstractQueryExecutor {
-
-  /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(InternalDetailWithOrderQueryExecutor.class.getName());
-
-  public InternalDetailWithOrderQueryExecutor() {
-    // numer of cores
-    try {
-      numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.NUM_CORES,
-              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
-    } catch (NumberFormatException e) {
-      numberOfCores = 1;
-    }
-    // get the number of records which can be kept in memory
-    // in case of detail query with order by we cannot keep all the
-    // record, so based on number of records which can be kept
-    // update the number of cores, this is required,
-    // if number of cores is 4 and number of records in memory is 120000
-    // then only one blocklet can be data can be kept in memory
-    int recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
-    String defaultInMemoryRecordsSize =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.INMEMORY_REOCRD_SIZE);
-    if (null != defaultInMemoryRecordsSize) {
-      try {
-        recordSize = Integer.parseInt(defaultInMemoryRecordsSize);
-      } catch (NumberFormatException ne) {
-        LOGGER.error("Invalid inmemory records size. Using default value");
-        recordSize = CarbonCommonConstants.INMEMORY_REOCRD_SIZE_DEFAULT;
-      }
-    }
-    this.numberOfCores = recordSize / Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL));
-    if (numberOfCores == 0) {
-      numberOfCores++;
-    }
-    execService = Executors.newFixedThreadPool(numberOfCores);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java
deleted file mode 100644
index 4fffaa0..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalFunctionQueryExecutor.java
+++ /dev/null
@@ -1,36 +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.carbondata.query.carbon.executor.internal.impl;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-
-/**
- * This is to handle function query like
- * count(1), as function query will be executed like
- * count start query so it is extending the count star executor
- */
-public class InternalFunctionQueryExecutor extends InternalCountStartQueryExecutor {
-
-  public InternalFunctionQueryExecutor(List<AbstractIndex> blockList) {
-    super(blockList);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
index bd5ab00..f341fa9 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/QueryRunner.java
@@ -27,7 +27,6 @@ import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.processor.BlockProcessor;
-import org.carbondata.query.carbon.processor.impl.AggregateQueryBlockProcessor;
 import org.carbondata.query.carbon.processor.impl.DetailQueryBlockProcessor;
 
 /**
@@ -58,11 +57,7 @@ public class QueryRunner implements Callable<Void> {
     this.fileReader = FileFactory.getFileHolder(executionInfo.getFileType());
     // if detail query detail query processor will be used to process the
     // block
-    if (executionInfo.isDetailQuery() || executionInfo.isRawRecordDetailQuery()) {
-      dataBlockProcessor = new DetailQueryBlockProcessor(executionInfo, fileReader);
-    } else {
-      dataBlockProcessor = new AggregateQueryBlockProcessor(executionInfo, fileReader);
-    }
+    dataBlockProcessor = new DetailQueryBlockProcessor(executionInfo, fileReader);
   }
 
   @Override public Void call() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
index 789f77e..516230d 100644
--- a/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/carbondata/query/carbon/executor/util/QueryUtil.java
@@ -25,7 +25,6 @@ import java.util.Comparator;
 import java.util.HashMap;
 import java.util.HashSet;
 import java.util.Iterator;
-import java.util.LinkedHashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
@@ -43,7 +42,6 @@ import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -51,15 +49,8 @@ import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
-import org.carbondata.query.carbon.aggregator.dimension.DimensionDataAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.ColumnGroupDimensionsAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.DirectDictionaryDimensionAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.FixedLengthDimensionAggregator;
-import org.carbondata.query.carbon.aggregator.dimension.impl.VariableLengthDimensionAggregator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.model.CustomAggregateExpression;
-import org.carbondata.query.carbon.model.DimensionAggregatorInfo;
 import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QueryMeasure;
 import org.carbondata.query.carbon.model.QueryModel;
@@ -161,7 +152,6 @@ public class QueryUtil {
    *
    * @param queryDimensions dimension selected in query
    * @param generator       key generator
-   * @param allDimension    all dimension present in the table
    * @return max key for dimension
    * @throws KeyGenException if any problem while generating the key
    */
@@ -203,29 +193,6 @@ public class QueryUtil {
   }
 
   /**
-   * Below method will be used to extract the dimension and measure from the
-   * expression
-   *
-   * @param expressions aggregate expression
-   * @param dims        extracted dimensions
-   * @param msrs        extracted measures
-   */
-  public static void extractDimensionsAndMeasuresFromExpression(
-      List<CustomAggregateExpression> expressions, List<CarbonDimension> dims,
-      List<CarbonMeasure> msrs) {
-    for (CustomAggregateExpression expression : expressions) {
-      List<CarbonColumn> dimsFromExpr = expression.getReferredColumns();
-      for (CarbonColumn dimFromExpr : dimsFromExpr) {
-        if (!dimFromExpr.isDimesion()) {
-          msrs.add((CarbonMeasure) dimFromExpr);
-        } else {
-          dims.add((CarbonDimension) dimFromExpr);
-        }
-      }
-    }
-  }
-
-  /**
    * Below method will be used to get the dimension block index in file based
    * on query dimension
    *
@@ -235,17 +202,13 @@ public class QueryUtil {
    */
   public static int[] getDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
       Map<Integer, Integer> dimensionOrdinalToBlockMapping,
-      List<DimensionAggregatorInfo> dimAggInfo, List<CarbonDimension> customAggregationDimension) {
+      List<CarbonDimension> customAggregationDimension) {
     // using set as in row group columns will point to same block
     Set<Integer> dimensionBlockIndex = new HashSet<Integer>();
     for (int i = 0; i < queryDimensions.size(); i++) {
       dimensionBlockIndex.add(
           dimensionOrdinalToBlockMapping.get(queryDimensions.get(i).getDimension().getOrdinal()));
     }
-    for (int i = 0; i < dimAggInfo.size(); i++) {
-      dimensionBlockIndex
-          .add(dimensionOrdinalToBlockMapping.get(dimAggInfo.get(i).getDim().getOrdinal()));
-    }
     for (int i = 0; i < customAggregationDimension.size(); i++) {
       dimensionBlockIndex
           .add(dimensionOrdinalToBlockMapping.get(customAggregationDimension.get(i).getOrdinal()));
@@ -260,18 +223,12 @@ public class QueryUtil {
    *
    * @param queryDimensions            query dimension present in the query this will be used to
    *                                   convert the result from surrogate key to actual data
-   * @param dimAggInfo                 dimension present in the dimension aggregation
-   *                                   dictionary will be used to convert to actual data
-   *                                   for aggregation
-   * @param customAggregationDimension dimension which is present in the expression for aggregation
-   *                                   we need dictionary data
    * @param absoluteTableIdentifier    absolute table identifier
    * @return dimension unique id to its dictionary map
    * @throws QueryExecutionException
    */
   public static Map<String, Dictionary> getDimensionDictionaryDetail(
-      List<QueryDimension> queryDimensions, List<DimensionAggregatorInfo> dimAggInfo,
-      List<CustomAggregateExpression> customAggExpression,
+      List<QueryDimension> queryDimensions,
       AbsoluteTableIdentifier absoluteTableIdentifier) throws QueryExecutionException {
     // to store dimension unique column id list, this is required as
     // dimension can be present in
@@ -287,23 +244,6 @@ public class QueryUtil {
         dictionaryDimensionFromQuery.add(queryDimensions.get(i).getDimension().getColumnId());
       }
     }
-    for (int i = 0; i < dimAggInfo.size(); i++) {
-      List<Encoding> encodingList = dimAggInfo.get(i).getDim().getEncoder();
-      if (CarbonUtil.hasEncoding(encodingList, Encoding.DICTIONARY) && !CarbonUtil
-          .hasEncoding(encodingList, Encoding.DIRECT_DICTIONARY)) {
-        dictionaryDimensionFromQuery.add(dimAggInfo.get(i).getDim().getColumnId());
-      }
-    }
-    for (int i = 0; i < customAggExpression.size(); i++) {
-      List<CarbonColumn> referredColumns = customAggExpression.get(i).getReferredColumns();
-      for (CarbonColumn column : referredColumns) {
-        if (CarbonUtil.hasEncoding(column.getEncoder(), Encoding.DICTIONARY) && !CarbonUtil
-            .hasEncoding(column.getEncoder(), Encoding.DIRECT_DICTIONARY)) {
-          dictionaryDimensionFromQuery.add(column.getColumnId());
-        }
-      }
-
-    }
     // converting to list as api exposed needed list which i think
     // is not correct
     List<String> dictionaryColumnIdList =
@@ -672,127 +612,6 @@ public class QueryUtil {
   }
 
   /**
-   * Below method will be used to get the dimension data aggregator list,
-   * which will be used to aggregate the dimension data, aggregate function
-   * like count(dimension) or any other function on dimension will be handled
-   *
-   * @param dimensionAggInfoList           dimension aggregation from query model
-   * @param dimensionToBlockIndexMapping   dimension block to its index mapping this will be
-   *                                       used to read  the block from file
-   * @param columnGroupIdToKeyGeneratorMap column group it to its key generator which will
-   *                                       be used to unpack the row group columns
-   * @param columnUniqueIdToDictionaryMap  this will dictionary column to its dictionary mapping
-   * @return list dimension data aggregator objects
-   */
-
-  public static List<DimensionDataAggregator> getDimensionDataAggregatorList1(
-      List<DimensionAggregatorInfo> dimensionAggInfoList,
-      Map<Integer, Integer> dimensionToBlockIndexMapping,
-      Map<Integer, KeyGenerator> columnGroupIdToKeyGeneratorMap,
-      Map<String, Dictionary> columnUniqueIdToDictionaryMap) {
-
-    Map<Integer, List<DimensionAggregatorInfo>> arrangeDimensionAggregationInfo =
-        arrangeDimensionAggregationInfo(dimensionAggInfoList, dimensionToBlockIndexMapping);
-    List<DimensionDataAggregator> dimensionDataAggregators =
-        new ArrayList<DimensionDataAggregator>();
-    int aggregatorStartIndex = 0;
-
-    for (Entry<Integer, List<DimensionAggregatorInfo>> entry : arrangeDimensionAggregationInfo
-        .entrySet()) {
-      // if number of dimension aggregation is info is more than 2 than it
-      // is a column group dimension
-      // so only one aggregator instance will be created to handle the
-      // aggregation
-      // as in case of column group unpacking of the bit packed mdkey will
-      // be done
-      // only once
-      CarbonDimension dim = entry.getValue().get(0).getDim();
-      if (entry.getValue().size() > 1) {
-        // how many aggregator will be used for column group
-        int numberOfAggregatorForColumnGroup = 0;
-        List<Dictionary> dictionaryList = new ArrayList<Dictionary>();
-        // below code is to create a dictionary list of all the column
-        // group dimension present in the query
-        for (DimensionAggregatorInfo dimensionAggregatorInfo : entry.getValue()) {
-          dictionaryList.add(
-              columnUniqueIdToDictionaryMap.get(dimensionAggregatorInfo.getDim().getColumnId()));
-          numberOfAggregatorForColumnGroup += dimensionAggregatorInfo.getAggList().size();
-        }
-        dimensionDataAggregators.add(new ColumnGroupDimensionsAggregator(entry.getValue(),
-            columnGroupIdToKeyGeneratorMap.get(dim.columnGroupId()),
-            dimensionToBlockIndexMapping.get(dim.getOrdinal()), dictionaryList,
-            aggregatorStartIndex));
-        aggregatorStartIndex += numberOfAggregatorForColumnGroup;
-        continue;
-      } else {
-        if(CarbonUtil.hasEncoding(dim.getEncoder(), Encoding.DIRECT_DICTIONARY)){
-          dimensionDataAggregators.add(
-              new DirectDictionaryDimensionAggregator(entry.getValue().get(0),
-                  aggregatorStartIndex,
-                  dimensionToBlockIndexMapping.get(dim.getOrdinal())));
-        }
-        // if it is a dictionary column than create a fixed length
-        // aggeragtor
-        else if (CarbonUtil
-            .hasEncoding(dim.getEncoder(), Encoding.DICTIONARY)) {
-          dimensionDataAggregators.add(
-              new FixedLengthDimensionAggregator(entry.getValue().get(0), null,
-                  columnUniqueIdToDictionaryMap.get(dim.getColumnId()),
-                  aggregatorStartIndex,
-                  dimensionToBlockIndexMapping.get(dim.getOrdinal())));
-        } else {
-          // else for not dictionary column create a
-          // variable length aggregator
-          dimensionDataAggregators.add(
-              new VariableLengthDimensionAggregator(entry.getValue().get(0), null,
-                  aggregatorStartIndex,
-                  dimensionToBlockIndexMapping.get(dim.getOrdinal())));
-        }
-        aggregatorStartIndex += entry.getValue().get(0).getAggList().size();
-      }
-    }
-    return dimensionDataAggregators;
-  }
-
-  /**
-   * Below method will be used to group the dimension aggregation infos This
-   * grouping will be based on block index of the file to dimension mapping
-   * Basically it will group all the dimension aggregation of same column
-   * group dimension This is done to avoid the column group dimension mdkey
-   * unpacking multiple times. If all the dimension of column group is handled
-   * separately then unpacking of mdkey for same column group will be done
-   * multiple times and it will impact the query performance, so to avoid this
-   * if we group the dimension together and they will point same block in the
-   * physical file so reading will be done only once and unpacking of each row
-   * will be also done only once
-   *
-   * @param queryDimensionAggregationInfos      query dimension aggregation infos
-   * @param dimensionOrdinaltoBlockIndexMapping dimension to file block index mapping
-   * @return block index in file to list of dimension pointing to that block
-   * mapping
-   */
-  private static Map<Integer, List<DimensionAggregatorInfo>> arrangeDimensionAggregationInfo(
-      List<DimensionAggregatorInfo> queryDimensionAggregationInfos,
-      Map<Integer, Integer> dimensionOrdinaltoBlockIndexMapping) {
-    Map<Integer, List<DimensionAggregatorInfo>> groupedDimensionAggregationInfo =
-        new LinkedHashMap<Integer, List<DimensionAggregatorInfo>>();
-    for (DimensionAggregatorInfo queryDimensionAggregatorInfo : queryDimensionAggregationInfos) {
-      List<DimensionAggregatorInfo> list = groupedDimensionAggregationInfo.get(
-          dimensionOrdinaltoBlockIndexMapping
-              .get(queryDimensionAggregatorInfo.getDim().getOrdinal()));
-
-      if (null == list) {
-        list =
-            new ArrayList<DimensionAggregatorInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-        groupedDimensionAggregationInfo.put(dimensionOrdinaltoBlockIndexMapping
-            .get(queryDimensionAggregatorInfo.getDim().getOrdinal()), list);
-      }
-      list.add(queryDimensionAggregatorInfo);
-    }
-    return groupedDimensionAggregationInfo;
-  }
-
-  /**
    * Below method will be used to fill block indexes of the query dimension
    * which will be used in creating a output row Here is method we are passing
    * two list which store the indexes one for dictionary column other for not
@@ -860,10 +679,6 @@ public class QueryUtil {
             .setMeasure(carbonTable.getMeasureByName(tableName, queryMeasure.getColumnName()));
       }
     }
-    // resolve dimension aggregation info
-    for (DimensionAggregatorInfo dimAggInfo : queryModel.getDimAggregationInfo()) {
-      dimAggInfo.setDim(carbonTable.getDimensionByName(tableName, dimAggInfo.getColumnName()));
-    }
     //TODO need to handle expression
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java b/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
index 8b77e45..9358f42 100644
--- a/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
+++ b/core/src/main/java/org/carbondata/query/carbon/merger/AbstractScannedResultMerger.java
@@ -33,7 +33,6 @@ import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.result.impl.ListBasedResult;
-import org.carbondata.query.carbon.result.impl.MapBasedResult;
 import org.carbondata.query.carbon.result.iterator.MemoryBasedResultIterator;
 
 /**
@@ -93,11 +92,7 @@ public abstract class AbstractScannedResultMerger implements ScannedResultMerger
    * for initializing the map based or list based result.
    */
   protected void initialiseResult() {
-    if (!blockExecutionInfo.isDetailQuery() && !blockExecutionInfo.isRawRecordDetailQuery()) {
-      mergedScannedResult = new MapBasedResult();
-    } else {
-      mergedScannedResult = new ListBasedResult();
-    }
+    mergedScannedResult = new ListBasedResult();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java b/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java
deleted file mode 100644
index 2ed565e..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/merger/impl/SortedScannedResultMerger.java
+++ /dev/null
@@ -1,176 +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.carbondata.query.carbon.merger.impl;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.executor.infos.SortInfo;
-import org.carbondata.query.carbon.merger.AbstractScannedResultMerger;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.comparator.FixedLengthKeyResultComparator;
-import org.carbondata.query.carbon.result.comparator.VariableLengthKeyResultComparator;
-import org.carbondata.query.carbon.result.iterator.MemoryBasedResultIterator;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-import org.apache.commons.collections.comparators.ComparatorChain;
-
-/**
- * Below class will be used to sort and merge the scanned result
- */
-public class SortedScannedResultMerger extends AbstractScannedResultMerger {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(SortedScannedResultMerger.class.getName());
-
-  public SortedScannedResultMerger(BlockExecutionInfo blockExecutionInfo,
-      int maxNumberOfScannedresultList) {
-    super(blockExecutionInfo, maxNumberOfScannedresultList);
-  }
-
-  /**
-   * Below method will be used to get the comparator for sorting the
-   * result
-   *
-   * @param sortInfo sort info
-   * @return comparator
-   */
-  public static ComparatorChain getMergerChainComparator(SortInfo sortInfo) {
-    List<Comparator> compratorList =
-        new ArrayList<Comparator>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    int length = sortInfo.getSortDimension().size();
-    int noDictionaryIndex = 0;
-    for (int i = 0; i < length; i++) {
-      if (!CarbonUtil.hasEncoding(sortInfo.getSortDimension().get(i).getDimension().getEncoder(),
-          Encoding.DICTIONARY)) {
-        compratorList.add(new VariableLengthKeyResultComparator(sortInfo.getDimensionSortOrder()[i],
-            noDictionaryIndex++, sortInfo.getSortDimension().get(i).getDimension().getDataType()));
-      } else {
-        compratorList.add(
-            new FixedLengthKeyResultComparator(sortInfo.getMaskedByteRangeForSorting()[i],
-                sortInfo.getDimensionSortOrder()[i], sortInfo.getDimensionMaskKeyForSorting()[i]));
-      }
-    }
-    return new ComparatorChain(compratorList);
-  }
-
-  /**
-   * Below method will be used to get the final query
-   * return
-   *
-   * @return iterator over result
-   */
-  @Override public CarbonIterator<Result> getQueryResultIterator() throws QueryExecutionException {
-    execService.shutdown();
-    try {
-      execService.awaitTermination(1, TimeUnit.DAYS);
-    } catch (InterruptedException e1) {
-      LOGGER.error("Problem in thread termination" + e1.getMessage());
-    }
-    if (scannedResultList.size() > 0) {
-      mergeScannedResults(scannedResultList);
-      scannedResultList = null;
-    }
-    LOGGER.debug("Finished result merging from all slices");
-    sortResult();
-    return new MemoryBasedResultIterator(mergedScannedResult);
-  }
-
-  /**
-   * Below method will be used to sort the query result
-   * for dictionary dimension it will unpack the key array a
-   * and then it will get the sort index based on which new dictionary
-   * key will be created
-   *
-   * @throws QueryExecutionException
-   */
-  private void sortResult() throws QueryExecutionException {
-    List<ListBasedResultWrapper> result =
-        new ArrayList<ListBasedResultWrapper>(mergedScannedResult.size());
-    ListBasedResultWrapper wrapper = null;
-    SortInfo sortInfo = blockExecutionInfo.getSortInfo();
-    KeyStructureInfo keyStructureInfo = blockExecutionInfo.getKeyStructureInfo();
-    long[] keyArray = null;
-    try {
-      while (mergedScannedResult.hasNext()) {
-        wrapper = new ListBasedResultWrapper();
-        ByteArrayWrapper key = mergedScannedResult.getKey();
-        keyArray = keyStructureInfo.getKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(), keyStructureInfo.getMaskedBytes());
-        for (int i = 0; i < sortInfo.getSortDimension().size(); i++) {
-          if (CarbonUtil.hasEncoding(sortInfo.getSortDimension().get(i).getDimension().getEncoder(),
-              Encoding.DICTIONARY)) {
-            keyArray[sortInfo.getSortDimension().get(i).getDimension().getKeyOrdinal()] =
-                blockExecutionInfo.getColumnIdToDcitionaryMapping()
-                    .get(sortInfo.getSortDimension().get(i).getDimension().getColumnId())
-                    .getSortedIndex((int) keyArray[sortInfo.getSortDimension().get(i).getDimension()
-                        .getKeyOrdinal()]);
-          }
-        }
-        key.setDictionaryKey(getMaskedKey(keyStructureInfo.getKeyGenerator().generateKey(keyArray),
-            keyStructureInfo));
-        wrapper.setKey(key);
-        wrapper.setValue(mergedScannedResult.getValue());
-        result.add(wrapper);
-      }
-    } catch (KeyGenException e) {
-      throw new QueryExecutionException(e);
-    }
-    initialiseResult();
-    Collections.sort(result, getMergerChainComparator(sortInfo));
-    mergedScannedResult.addScannedResult(result);
-  }
-
-  /**
-   * Below method will be used to get the masked key
-   *
-   * @param data
-   * @return keyStructureInfo
-   */
-  private byte[] getMaskedKey(byte[] data, KeyStructureInfo keyStructureInfo) {
-    int keySize = blockExecutionInfo.getFixedLengthKeySize();
-    int[] actualMaskByteRanges = keyStructureInfo.getMaskByteRanges();
-    byte[] maxKey = keyStructureInfo.getMaxKey();
-    byte[] maskedKey = new byte[keySize];
-    int counter = 0;
-    int byteRange = 0;
-    for (int i = 0; i < keySize; i++) {
-      byteRange = actualMaskByteRanges[i];
-      maskedKey[counter++] = (byte) (data[byteRange] & maxKey[byteRange]);
-    }
-    return maskedKey;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java b/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
index dbff0c3..3684f4a 100644
--- a/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
+++ b/core/src/main/java/org/carbondata/query/carbon/model/CarbonQueryPlan.java
@@ -24,9 +24,7 @@ package org.carbondata.query.carbon.model;
 
 import java.io.Serializable;
 import java.util.ArrayList;
-import java.util.LinkedHashMap;
 import java.util.List;
-import java.util.Map;
 
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.query.expression.Expression;
@@ -99,12 +97,6 @@ public class CarbonQueryPlan implements Serializable {
   private String outLocationPath;
 
   /**
-   * dimAggregatorInfoList
-   */
-  private Map<String, DimensionAggregatorInfo> dimAggregatorInfos =
-      new LinkedHashMap<String, DimensionAggregatorInfo>();
-
-  /**
    * isCountStarQuery
    */
   private boolean isCountStartQuery;
@@ -225,24 +217,6 @@ public class CarbonQueryPlan implements Serializable {
     this.outLocationPath = outLocationPath;
   }
 
-  public void addAggDimAggInfo(String columnName, String aggType, int queryOrder) {
-    DimensionAggregatorInfo dimensionAggregatorInfo = dimAggregatorInfos.get(columnName);
-    if (null == dimensionAggregatorInfo) {
-      dimensionAggregatorInfo = new DimensionAggregatorInfo();
-      dimensionAggregatorInfo.setColumnName(columnName);
-      List<Integer> queryOrderList= new ArrayList<Integer>();
-      queryOrderList.add(queryOrder);
-      List<String> aggTypeList= new ArrayList<String>();
-      aggTypeList.add(aggType);
-      dimensionAggregatorInfo.setOrderList(queryOrderList);
-      dimensionAggregatorInfo.setAggList(aggTypeList);
-      dimAggregatorInfos.put(columnName, dimensionAggregatorInfo);
-    } else {
-      dimensionAggregatorInfo.getOrderList().add(queryOrder);
-      dimensionAggregatorInfo.getAggList().add(aggType);
-    }
-  }
-
   public boolean isCountStarQuery() {
     return isCountStartQuery;
   }
@@ -251,10 +225,6 @@ public class CarbonQueryPlan implements Serializable {
     this.isCountStartQuery = isCountStartQuery;
   }
 
-  public Map<String, DimensionAggregatorInfo> getDimAggregatorInfos() {
-    return dimAggregatorInfos;
-  }
-
   public List<QueryDimension> getSortedDimemsions() {
     return sortedDimensions;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java b/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java
deleted file mode 100644
index 9890f7f..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/CustomAggregateExpression.java
+++ /dev/null
@@ -1,149 +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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.CustomMeasureAggregator;
-
-/**
- * Holds the information about expression present in the query
- */
-public class CustomAggregateExpression implements Serializable {
-
-  /**
-   * serialization version
-   */
-  private static final long serialVersionUID = 4831882661337567735L;
-
-  /**
-   * Identified and delegated from Spark Layer for UDAF in Carbon
-   */
-  private CustomMeasureAggregator aggregator;
-
-  /**
-   * Aggregate column name may not be a measure or dimension. Can be a column
-   * name given in query
-   */
-  private String name;
-
-  /**
-   * Columns used in the expression where column can be a dimension or a
-   * measure.
-   */
-  private List<CarbonColumn> referredColumns;
-
-  /**
-   * Actual expression in query to use in the comparison with other Aggregate
-   * expressions.
-   */
-  private String expression;
-
-  /**
-   * Position in the query
-   */
-  private int queryOrder;
-
-  public CustomAggregateExpression() {
-    referredColumns = new ArrayList<CarbonColumn>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  public String getExpression() {
-    return expression;
-  }
-
-  public void setExpression(String expression) {
-    this.expression = expression;
-  }
-
-  public List<CarbonColumn> getReferredColumns() {
-    return referredColumns;
-  }
-
-  public void setReferredColumns(List<CarbonColumn> referredColumns) {
-    this.referredColumns = referredColumns;
-  }
-
-  public int getQueryOrder() {
-    return queryOrder;
-  }
-
-  public void setQueryOrder(int queryOrder) {
-    this.queryOrder = queryOrder;
-  }
-
-  public String getName() {
-    return name;
-  }
-
-  public void setName(String name) {
-    this.name = name;
-  }
-
-  /**
-   * @return MeasureAggregator
-   */
-  public CustomMeasureAggregator getAggregator() {
-    return aggregator;
-  }
-
-  /**
-   * @param aggregator
-   */
-  public void setAggregator(CustomMeasureAggregator aggregator) {
-    this.aggregator = aggregator;
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    result = prime * result + ((expression == null) ? 0 : expression.hashCode());
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (obj == this) {
-      return true;
-    }
-
-    if (!(obj instanceof CustomAggregateExpression)) {
-      return false;
-    }
-
-    CustomAggregateExpression other = ((CustomAggregateExpression) obj);
-
-    if ((expression != null) && (expression.equals(other.expression))) {
-      return true;
-    }
-
-    if (expression != null) {
-      return expression.equalsIgnoreCase(other.expression);
-    }
-
-    if (other.expression != null) {
-      return other.expression.equalsIgnoreCase(expression);
-    }
-
-    return true;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
index 57d96d0..64b519a 100644
--- a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
+++ b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
@@ -31,7 +31,6 @@ import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.expression.ColumnExpression;
 import org.carbondata.query.expression.Expression;
@@ -88,16 +87,6 @@ public class QueryModel implements Serializable {
   private int limit;
 
   /**
-   * for applying aggregation on dimension
-   */
-  private List<DimensionAggregatorInfo> dimAggregationInfo;
-
-  /**
-   * custom aggregate expression
-   */
-  private List<CustomAggregateExpression> expressions;
-
-  /**
    * to check if it is a count star query , so processing will be different
    */
   private boolean isCountStarQuery;
@@ -153,10 +142,6 @@ public class QueryModel implements Serializable {
 
   public QueryModel() {
     tableBlockInfos = new ArrayList<TableBlockInfo>();
-    dimAggregationInfo =
-        new ArrayList<DimensionAggregatorInfo>();
-    expressions =
-        new ArrayList<CustomAggregateExpression>();
     queryDimension = new ArrayList<QueryDimension>();
     queryMeasures = new ArrayList<QueryMeasure>();
     sortDimension = new ArrayList<QueryDimension>();
@@ -173,7 +158,6 @@ public class QueryModel implements Serializable {
 
     fillQueryModel(queryPlan, carbonTable, queryModel, factTableName);
 
-    fillDimensionAggregator(queryPlan, queryModel);
     queryModel.setLimit(queryPlan.getLimit());
     queryModel.setDetailQuery(queryPlan.isDetailQuery());
     queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery());
@@ -217,17 +201,6 @@ public class QueryModel implements Serializable {
 
   }
 
-  private static void fillDimensionAggregator(CarbonQueryPlan logicalPlan,
-      QueryModel executorModel) {
-    Map<String, DimensionAggregatorInfo> dimAggregatorInfos = logicalPlan.getDimAggregatorInfos();
-    List<DimensionAggregatorInfo> dimensionAggregatorInfos =
-        new ArrayList<DimensionAggregatorInfo>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    for (Map.Entry<String, DimensionAggregatorInfo> entry : dimAggregatorInfos.entrySet()) {
-      dimensionAggregatorInfos.add(entry.getValue());
-    }
-    executorModel.setDimAggregationInfo(dimensionAggregatorInfos);
-  }
-
   public static void processFilterExpression(
       Expression filterExpression, List<CarbonDimension> dimensions, List<CarbonMeasure> measures) {
     if (null != filterExpression) {
@@ -397,20 +370,6 @@ public class QueryModel implements Serializable {
   }
 
   /**
-   * @return the dimAggregationInfo
-   */
-  public List<DimensionAggregatorInfo> getDimAggregationInfo() {
-    return dimAggregationInfo;
-  }
-
-  /**
-   * @param dimAggregationInfo the dimAggregationInfo to set
-   */
-  public void setDimAggregationInfo(List<DimensionAggregatorInfo> dimAggregationInfo) {
-    this.dimAggregationInfo = dimAggregationInfo;
-  }
-
-  /**
    * @return the tableBlockInfos
    */
   public List<TableBlockInfo> getTableBlockInfos() {
@@ -425,20 +384,6 @@ public class QueryModel implements Serializable {
   }
 
   /**
-   * @return the expressions
-   */
-  public List<CustomAggregateExpression> getExpressions() {
-    return expressions;
-  }
-
-  /**
-   * @param expressions the expressions to set
-   */
-  public void setExpressions(List<CustomAggregateExpression> expressions) {
-    this.expressions = expressions;
-  }
-
-  /**
    * @return the queryTempLocation
    */
   public String getQueryTempLocation() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
index 392fcd7..fdcf6f1 100644
--- a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
+++ b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockProcessor.java
@@ -23,10 +23,8 @@ import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.DataRefNode;
 import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.aggregator.DataAggregator;
 import org.carbondata.query.carbon.aggregator.ScannedResultAggregator;
 import org.carbondata.query.carbon.aggregator.impl.ListBasedResultAggregator;
-import org.carbondata.query.carbon.aggregator.impl.MapBasedResultAggregator;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
 import org.carbondata.query.carbon.scanner.BlockletScanner;
@@ -82,13 +80,8 @@ public abstract class AbstractDataBlockProcessor implements BlockProcessor {
       blockletScanner = new NonFilterScanner(blockExecutionInfo);
     }
 
-    if (blockExecutionInfo.isDetailQuery() || blockExecutionInfo.isRawRecordDetailQuery()) {
-      this.scannerResultAggregator =
-          new ListBasedResultAggregator(blockExecutionInfo, new DataAggregator(blockExecutionInfo));
-    } else {
-      this.scannerResultAggregator =
-          new MapBasedResultAggregator(blockExecutionInfo, new DataAggregator(blockExecutionInfo));
-    }
+    this.scannerResultAggregator =
+        new ListBasedResultAggregator(blockExecutionInfo);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
index 81d09ae..444b2bd 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
@@ -135,6 +135,14 @@ public abstract class AbstractScannedResult {
   }
 
   /**
+   * Just increment the counter incase of query only on measures.
+   */
+  public void incrementCounter() {
+    rowCounter ++;
+    currentRow ++;
+  }
+
+  /**
    * Below method will be used to get the dimension data based on dimension
    * ordinal and index
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
index 26588e6..4aa2adf 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
@@ -20,10 +20,8 @@
 package org.carbondata.query.carbon.result;
 
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.carbon.model.QueryDimension;
 import org.carbondata.query.carbon.model.QuerySchemaInfo;
 import org.carbondata.query.carbon.util.DataTypeUtil;
@@ -32,38 +30,9 @@ import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 /**
  * Below class holds the query result of batches.
  */
-public class BatchRawResult extends CarbonIterator<Object[]> {
-
-  /**
-   * list of keys
-   */
-  private Object[][] rows;
+public class BatchRawResult extends BatchResult {
 
   private QuerySchemaInfo querySchemaInfo;
-  /**
-   * counter to check whether all the records are processed or not
-   */
-  private int counter;
-
-  /**
-   * size of the batches.
-   */
-  private int size;
-
-  public BatchRawResult(Object[][] rows) {
-    this.rows = rows;
-    if (rows.length > 0) {
-      this.size = rows[0].length;
-    }
-  }
-
-  /**
-   * This will return all the raw records.
-   * @return
-   */
-  public Object[][] getAllRows() {
-    return rows;
-  }
 
   /**
    * This method will return one row at a time based on the counter given.
@@ -71,22 +40,8 @@ public class BatchRawResult extends CarbonIterator<Object[]> {
    * @return
    */
   public Object[] getRawRow(int counter) {
-    Object[] outputRow = new Object[rows.length];
-    for(int col = 0 ; col < rows.length ; col++) {
-      outputRow[col] = rows[col][counter];
-    }
-    return outputRow;
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements.
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return counter < size;
+    return rows[counter];
   }
-
   /**
    * Returns the next element in the iteration.
    *
@@ -97,71 +52,38 @@ public class BatchRawResult extends CarbonIterator<Object[]> {
   }
 
   private Object[] parseData() {
-    ByteArrayWrapper key = (ByteArrayWrapper) rows[0][counter];
     int[] order = querySchemaInfo.getQueryReverseOrder();
-    long[] surrogateResult = querySchemaInfo.getKeyGenerator()
-        .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
+    Object[] row = rows[counter];
+    ByteArrayWrapper key = (ByteArrayWrapper) row[0];
     QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
-    Object[] parsedData = new Object[queryDimensions.length + rows.length - 1];
-    int noDictionaryColumnIndex = 0;
-    for (int i = 0; i < queryDimensions.length; i++) {
-      if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
-        parsedData[order[i]] = DataTypeUtil.getDataBasedOnDataType(
-            new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
-            queryDimensions[i].getDimension().getDataType());
-      } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
-        parsedData[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
-            (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
-      } else {
-        parsedData[order[i]] =
-            (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
+    Object[] parsedData = new Object[queryDimensions.length + row.length - 1];
+    if(key != null) {
+      long[] surrogateResult = querySchemaInfo.getKeyGenerator()
+          .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
+      int noDictionaryColumnIndex = 0;
+      for (int i = 0; i < queryDimensions.length; i++) {
+        if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
+          parsedData[order[i]] = DataTypeUtil.getDataBasedOnDataType(
+              new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
+              queryDimensions[i].getDimension().getDataType());
+        } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+          DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+              .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
+          parsedData[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
+              (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
+        } else {
+          parsedData[order[i]] =
+              (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
+        }
       }
     }
-    for (int i = 0; i < rows.length - 1; i++) {
-      parsedData[order[i + queryDimensions.length]] = rows[i + 1][counter];
+    for (int i = 0; i < row.length - 1; i++) {
+      parsedData[order[i + queryDimensions.length]] = row[i + 1];
     }
     counter++;
     return parsedData;
   }
 
-  public static Object[] parseData(ByteArrayWrapper key, Object[] aggData,
-      QuerySchemaInfo querySchemaInfo, int[] aggOrder) {
-    long[] surrogateResult = querySchemaInfo.getKeyGenerator()
-        .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
-    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
-    Object[] parsedData = new Object[queryDimensions.length + aggData.length];
-    int noDictionaryColumnIndex = 0;
-    for (int i = 0; i < queryDimensions.length; i++) {
-      if (!CarbonUtil
-          .hasEncoding(queryDimensions[i].getDimension().getEncoder(), Encoding.DICTIONARY)) {
-        parsedData[i] = DataTypeUtil.getDataBasedOnDataType(
-            new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
-            queryDimensions[i].getDimension().getDataType());
-      } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(queryDimensions[i].getDimension().getDataType());
-        parsedData[i] = directDictionaryGenerator.getValueFromSurrogate(
-            (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
-      } else {
-        parsedData[i] = (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
-      }
-    }
-    for (int i = 0; i < aggData.length; i++) {
-      parsedData[i + queryDimensions.length] = aggData[i];
-    }
-    Object[] orderData = new Object[parsedData.length];
-    for (int i = 0; i < parsedData.length; i++) {
-      orderData[i] = parsedData[aggOrder[i]];
-    }
-    return orderData;
-  }
-
-  public QuerySchemaInfo getQuerySchemaInfo() {
-    return querySchemaInfo;
-  }
-
   public void setQuerySchemaInfo(QuerySchemaInfo querySchemaInfo) {
     this.querySchemaInfo = querySchemaInfo;
   }
@@ -171,6 +93,6 @@ public class BatchRawResult extends CarbonIterator<Object[]> {
    * @return
    */
   public int getSize() {
-    return size;
+    return rows.length;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java b/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
index f8e2a54..220336b 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
@@ -19,82 +19,55 @@
 
 package org.carbondata.query.carbon.result;
 
-import java.util.ArrayList;
-import java.util.List;
+import java.util.NoSuchElementException;
 
-import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.scanner.impl.CarbonKey;
-import org.carbondata.query.scanner.impl.CarbonValue;
 
 /**
  * Below class holds the query result
  */
-public class BatchResult extends CarbonIterator<RowResult> {
+public class BatchResult extends CarbonIterator<Object[]> {
 
   /**
    * list of keys
    */
-  private List<CarbonKey> keys;
-
-  /**
-   * list of values
-   */
-  private List<CarbonValue> values;
+  protected Object[][] rows;
 
   /**
    * counter to check whether all the records are processed or not
    */
-  private int counter;
+  protected int counter;
 
   public BatchResult() {
-    keys = new ArrayList<CarbonKey>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    values = new ArrayList<CarbonValue>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    this.rows = new Object[0][];
   }
 
   /**
-   * Below method will be used to get the key
+   * Below method will be used to get the rows
    *
    * @return
    */
-  public List<CarbonKey> getKeys() {
-    return keys;
-  }
-
-  /**
-   * below method will be used to set the key
-   *
-   * @param keys
-   */
-  public void setKeys(List<CarbonKey> keys) {
-    this.keys = keys;
-  }
-
-  /**
-   * Below method will be used to get the values
-   *
-   * @return
-   */
-  public List<CarbonValue> getValues() {
-    return values;
+  public Object[][] getRows() {
+    return rows;
   }
 
   /**
    * Below method will be used to get the set the values
    *
-   * @param values
+   * @param rows
    */
-  public void setValues(List<CarbonValue> values) {
-    this.values = values;
+  public void setRows(Object[][] rows) {
+    this.rows = rows;
   }
 
+
   /**
    * Returns {@code true} if the iteration has more elements.
    *
    * @return {@code true} if the iteration has more elements
    */
   @Override public boolean hasNext() {
-    return counter < keys.size();
+    return counter < rows.length;
   }
 
   /**
@@ -102,11 +75,12 @@ public class BatchResult extends CarbonIterator<RowResult> {
    *
    * @return the next element in the iteration
    */
-  @Override public RowResult next() {
-    RowResult rowResult = new RowResult();
-    rowResult.setKey(keys.get(counter));
-    rowResult.setValue(values.get(counter));
+  @Override public Object[] next() {
+    if (!hasNext()) {
+      throw new NoSuchElementException();
+    }
+    Object[] row = rows[counter];
     counter++;
-    return rowResult;
+    return row;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java b/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
index ca72d2d..1a40fc3 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
@@ -1,13 +1,12 @@
 package org.carbondata.query.carbon.result;
 
-import org.carbondata.query.aggregator.MeasureAggregator;
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 
 public class ListBasedResultWrapper {
 
   private ByteArrayWrapper key;
 
-  private MeasureAggregator[] value;
+  private Object[] value;
 
   /**
    * @return the key
@@ -26,14 +25,14 @@ public class ListBasedResultWrapper {
   /**
    * @return the value
    */
-  public MeasureAggregator[] getValue() {
+  public Object[] getValue() {
     return value;
   }
 
   /**
    * @param value the value to set
    */
-  public void setValue(MeasureAggregator[] value) {
+  public void setValue(Object[] value) {
     this.value = value;
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/Result.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/Result.java b/core/src/main/java/org/carbondata/query/carbon/result/Result.java
index d4ba14f..8526b07 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/Result.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/Result.java
@@ -19,13 +19,12 @@
 
 package org.carbondata.query.carbon.result;
 
-import org.carbondata.query.aggregator.MeasureAggregator;
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 
 /**
  * Result interface for storing the result
  */
-public interface Result<K> {
+public interface Result<K, V> {
   /**
    * Below method will be used to
    * add the sccaed result
@@ -53,9 +52,9 @@ public interface Result<K> {
    *
    * @return value
    */
-  MeasureAggregator[] getValue();
+  V[] getValue();
 
-  void merge(Result<K> otherResult);
+  void merge(Result<K, V> otherResult);
 
   /**
    * Below method will be used to get the result

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java b/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java
deleted file mode 100644
index 93a9102..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/RowResult.java
+++ /dev/null
@@ -1,45 +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.carbondata.query.carbon.result;
-
-import org.carbondata.query.scanner.impl.CarbonKey;
-import org.carbondata.query.scanner.impl.CarbonValue;
-
-public class RowResult {
-  private CarbonKey key;
-
-  private CarbonValue value;
-
-  public CarbonKey getKey() {
-    return key;
-  }
-
-  public void setKey(CarbonKey key) {
-    this.key = key;
-  }
-
-  public CarbonValue getValue() {
-    return value;
-  }
-
-  public void setValue(CarbonValue value) {
-    this.value = value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6288ec71/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
index 8b7904a..3f31028 100644
--- a/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
+++ b/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
@@ -23,7 +23,6 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.aggregator.MeasureAggregator;
 import org.carbondata.query.carbon.result.ListBasedResultWrapper;
 import org.carbondata.query.carbon.result.Result;
 import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
@@ -31,7 +30,7 @@ import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
 /**
  * Below class is a holder over list based result wrapper
  */
-public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
+public class ListBasedResult implements Result<List<ListBasedResultWrapper>, Object> {
 
   /**
    * current result list
@@ -71,9 +70,9 @@ public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
 
   public ListBasedResult() {
     currentRowPointer =
-        new ArrayList<ListBasedResultWrapper>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     allRowsResult =
-        new ArrayList<List<ListBasedResultWrapper>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
   }
 
   /**
@@ -121,7 +120,7 @@ public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
   /**
    * @return will return the value
    */
-  @Override public MeasureAggregator[] getValue() {
+  @Override public Object[] getValue() {
     return currentRowPointer.get(listRecordCounter).getValue();
   }
 
@@ -131,7 +130,7 @@ public class ListBasedResult implements Result<List<ListBasedResultWrapper>> {
    *
    * @param otherResult return to be merged
    */
-  @Override public void merge(Result<List<ListBasedResultWrapper>> otherResult) {
+  @Override public void merge(Result<List<ListBasedResultWrapper>, Object> otherResult) {
     if (otherResult.size() > 0) {
       totalNumberOfRecords += otherResult.size();
       this.allRowsResult.add(otherResult.getResult());



[31/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/infos/SortInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/infos/SortInfo.java b/core/src/main/java/org/carbondata/scan/executor/infos/SortInfo.java
new file mode 100644
index 0000000..53584f5
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/infos/SortInfo.java
@@ -0,0 +1,125 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.executor.infos;
+
+import java.util.List;
+
+import org.carbondata.scan.model.QueryDimension;
+
+/**
+ * Below class holds the order by information about the query
+ */
+public class SortInfo {
+
+  /**
+   * sorting order of a dimension
+   */
+  private byte[] dimensionSortOrder;
+
+  /**
+   * byte range of each dimension present in the order by
+   */
+  private int[][] maskedByteRangeForSorting;
+
+  /**
+   * dimension indexes which is used in order bye
+   */
+  private byte[] sortDimensionIndex;
+
+  /**
+   * mask key of each dimension
+   * this will be used to sort the dimension
+   */
+  private byte[][] dimensionMaskKeyForSorting;
+
+  /**
+   * sortDimension
+   */
+  private List<QueryDimension> sortDimension;
+
+  /**
+   * @return the dimensionSortOrder
+   */
+  public byte[] getDimensionSortOrder() {
+    return dimensionSortOrder;
+  }
+
+  /**
+   * @param dimensionSortOrder the dimensionSortOrder to set
+   */
+  public void setDimensionSortOrder(byte[] dimensionSortOrder) {
+    this.dimensionSortOrder = dimensionSortOrder;
+  }
+
+  /**
+   * @return the maskedByteRangeForSorting
+   */
+  public int[][] getMaskedByteRangeForSorting() {
+    return maskedByteRangeForSorting;
+  }
+
+  /**
+   * @param maskedByteRangeForSorting the maskedByteRangeForSorting to set
+   */
+  public void setMaskedByteRangeForSorting(int[][] maskedByteRangeForSorting) {
+    this.maskedByteRangeForSorting = maskedByteRangeForSorting;
+  }
+
+  /**
+   * @return the sortDimensionIndex
+   */
+  public byte[] getSortDimensionIndex() {
+    return sortDimensionIndex;
+  }
+
+  /**
+   * @param sortDimensionIndex the sortDimensionIndex to set
+   */
+  public void setSortDimensionIndex(byte[] sortDimensionIndex) {
+    this.sortDimensionIndex = sortDimensionIndex;
+  }
+
+  /**
+   * @return the dimensionMaskKeyForSorting
+   */
+  public byte[][] getDimensionMaskKeyForSorting() {
+    return dimensionMaskKeyForSorting;
+  }
+
+  /**
+   * @param dimensionMaskKeyForSorting the dimensionMaskKeyForSorting to set
+   */
+  public void setDimensionMaskKeyForSorting(byte[][] dimensionMaskKeyForSorting) {
+    this.dimensionMaskKeyForSorting = dimensionMaskKeyForSorting;
+  }
+
+  /**
+   * @return the sortDimension
+   */
+  public List<QueryDimension> getSortDimension() {
+    return sortDimension;
+  }
+
+  /**
+   * @param sortDimension the sortDimension to set
+   */
+  public void setSortDimension(List<QueryDimension> sortDimension) {
+    this.sortDimension = sortDimension;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java b/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
new file mode 100644
index 0000000..a7d98a3
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
@@ -0,0 +1,718 @@
+/*
+ * 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.carbondata.scan.executor.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.HashMap;
+import java.util.HashSet;
+import java.util.Iterator;
+import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
+import java.util.TreeSet;
+
+import org.carbondata.core.cache.Cache;
+import org.carbondata.core.cache.CacheProvider;
+import org.carbondata.core.cache.CacheType;
+import org.carbondata.core.cache.dictionary.Dictionary;
+import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
+import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.CarbonMetadata;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.core.keygenerator.KeyGenerator;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.core.util.CarbonUtilException;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.model.QueryModel;
+
+import org.apache.commons.lang3.ArrayUtils;
+
+/**
+ * Utility class for query execution
+ */
+public class QueryUtil {
+
+  /**
+   * Below method will be used to get the masked byte range based on the query
+   * dimension. It will give the range in the mdkey. This will be used to get
+   * the actual key array from masked mdkey
+   *
+   * @param queryDimensions query dimension selected in query
+   * @param keyGenerator    key generator
+   * @return masked key
+   */
+  public static int[] getMaskedByteRange(List<QueryDimension> queryDimensions,
+      KeyGenerator keyGenerator) {
+    Set<Integer> byteRangeSet = new TreeSet<Integer>();
+    int[] byteRange = null;
+    for (int i = 0; i < queryDimensions.size(); i++) {
+
+      // as no dictionary column and complex type columns
+      // are not selected in the mdkey
+      // so we will not select the those dimension for calculating the
+      // range
+      if (queryDimensions.get(i).getDimension().getKeyOrdinal() == -1) {
+        continue;
+      }
+      // get the offset of the dimension in the mdkey
+      byteRange =
+          keyGenerator.getKeyByteOffsets(queryDimensions.get(i).getDimension().getKeyOrdinal());
+      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
+        byteRangeSet.add(j);
+      }
+    }
+    int[] maksedByteRange = new int[byteRangeSet.size()];
+    int index = 0;
+    Iterator<Integer> iterator = byteRangeSet.iterator();
+    // add the masked byte range
+    while (iterator.hasNext()) {
+      maksedByteRange[index++] = iterator.next();
+    }
+    return maksedByteRange;
+  }
+
+  public static int[] getMaskedByteRangeBasedOrdinal(List<Integer> ordinals,
+      KeyGenerator keyGenerator) {
+    Set<Integer> byteRangeSet = new TreeSet<Integer>();
+    int[] byteRange = null;
+    for (int i = 0; i < ordinals.size(); i++) {
+
+      // get the offset of the dimension in the mdkey
+      byteRange = keyGenerator.getKeyByteOffsets(ordinals.get(i));
+      for (int j = byteRange[0]; j <= byteRange[1]; j++) {
+        byteRangeSet.add(j);
+      }
+    }
+    int[] maksedByteRange = new int[byteRangeSet.size()];
+    int index = 0;
+    Iterator<Integer> iterator = byteRangeSet.iterator();
+    // add the masked byte range
+    while (iterator.hasNext()) {
+      maksedByteRange[index++] = iterator.next();
+    }
+    return maksedByteRange;
+  }
+
+  /**
+   * Below method will return the max key based on the dimension ordinal
+   *
+   * @param keyOrdinalList
+   * @param generator
+   * @return
+   * @throws KeyGenException
+   */
+  public static byte[] getMaxKeyBasedOnOrinal(List<Integer> keyOrdinalList, KeyGenerator generator)
+      throws KeyGenException {
+    long[] max = new long[generator.getDimCount()];
+    Arrays.fill(max, 0L);
+
+    for (int i = 0; i < keyOrdinalList.size(); i++) {
+      // adding for dimension which is selected in query
+      max[keyOrdinalList.get(i)] = Long.MAX_VALUE;
+    }
+    return generator.generateKey(max);
+  }
+
+  /**
+   * To get the max key based on dimensions. i.e. all other dimensions will be
+   * set to 0 bits and the required query dimension will be masked with all
+   * LONG.MAX so that we can mask key and then compare while aggregating This
+   * can be useful during filter query when only few dimensions were selected
+   * out of row group
+   *
+   * @param queryDimensions dimension selected in query
+   * @param generator       key generator
+   * @return max key for dimension
+   * @throws KeyGenException if any problem while generating the key
+   */
+  public static byte[] getMaxKeyBasedOnDimensions(List<QueryDimension> queryDimensions,
+      KeyGenerator generator) throws KeyGenException {
+    long[] max = new long[generator.getDimCount()];
+    Arrays.fill(max, 0L);
+
+    for (int i = 0; i < queryDimensions.size(); i++) {
+      // as no dictionary column and complex type columns
+      // are not selected in the mdkey
+      // so we will not select the those dimension for calculating the
+      // range
+      if (queryDimensions.get(i).getDimension().getKeyOrdinal() == -1) {
+        continue;
+      }
+      // adding for dimension which is selected in query
+      max[queryDimensions.get(i).getDimension().getKeyOrdinal()] = Long.MAX_VALUE;
+    }
+
+    return generator.generateKey(max);
+  }
+
+  /**
+   * Below method will be used to get the masked key for query
+   *
+   * @param keySize         size of the masked key
+   * @param maskedKeyRanges masked byte range
+   * @return masked bytes
+   */
+  public static int[] getMaskedByte(int keySize, int[] maskedKeyRanges) {
+    int[] maskedKey = new int[keySize];
+    // all the non selected dimension will be filled with -1
+    Arrays.fill(maskedKey, -1);
+    for (int i = 0; i < maskedKeyRanges.length; i++) {
+      maskedKey[maskedKeyRanges[i]] = i;
+    }
+    return maskedKey;
+  }
+
+  /**
+   * Below method will be used to get the dimension block index in file based
+   * on query dimension
+   *
+   * @param queryDimensions                query dimension
+   * @param dimensionOrdinalToBlockMapping mapping of dimension block in file to query dimension
+   * @return block index of file
+   */
+  public static int[] getDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
+      Map<Integer, Integer> dimensionOrdinalToBlockMapping,
+      List<CarbonDimension> customAggregationDimension) {
+    // using set as in row group columns will point to same block
+    Set<Integer> dimensionBlockIndex = new HashSet<Integer>();
+    for (int i = 0; i < queryDimensions.size(); i++) {
+      dimensionBlockIndex.add(
+          dimensionOrdinalToBlockMapping.get(queryDimensions.get(i).getDimension().getOrdinal()));
+    }
+    for (int i = 0; i < customAggregationDimension.size(); i++) {
+      dimensionBlockIndex
+          .add(dimensionOrdinalToBlockMapping.get(customAggregationDimension.get(i).getOrdinal()));
+    }
+    return ArrayUtils
+        .toPrimitive(dimensionBlockIndex.toArray(new Integer[dimensionBlockIndex.size()]));
+  }
+
+  /**
+   * Below method will be used to get the dictionary mapping for all the
+   * dictionary encoded dimension present in the query
+   *
+   * @param queryDimensions            query dimension present in the query this will be used to
+   *                                   convert the result from surrogate key to actual data
+   * @param absoluteTableIdentifier    absolute table identifier
+   * @return dimension unique id to its dictionary map
+   * @throws QueryExecutionException
+   */
+  public static Map<String, Dictionary> getDimensionDictionaryDetail(
+      List<QueryDimension> queryDimensions,
+      AbsoluteTableIdentifier absoluteTableIdentifier) throws QueryExecutionException {
+    // to store dimension unique column id list, this is required as
+    // dimension can be present in
+    // query dimension, as well as some aggregation function will be applied
+    // in the same dimension
+    // so we need to get only one instance of dictionary
+    // direct dictionary skip is done only for the dictionary lookup
+    Set<String> dictionaryDimensionFromQuery = new HashSet<String>();
+    for (int i = 0; i < queryDimensions.size(); i++) {
+      List<Encoding> encodingList = queryDimensions.get(i).getDimension().getEncoder();
+      if (CarbonUtil.hasEncoding(encodingList, Encoding.DICTIONARY) && !CarbonUtil
+          .hasEncoding(encodingList, Encoding.DIRECT_DICTIONARY)) {
+        dictionaryDimensionFromQuery.add(queryDimensions.get(i).getDimension().getColumnId());
+      }
+    }
+    // converting to list as api exposed needed list which i think
+    // is not correct
+    List<String> dictionaryColumnIdList =
+        new ArrayList<String>(dictionaryDimensionFromQuery.size());
+    dictionaryColumnIdList.addAll(dictionaryDimensionFromQuery);
+    return getDictionaryMap(dictionaryColumnIdList, absoluteTableIdentifier);
+  }
+
+  /**
+   * Below method will be used to get the column id to its dictionary mapping
+   *
+   * @param dictionaryColumnIdList  dictionary column list
+   * @param absoluteTableIdentifier absolute table identifier
+   * @return dictionary mapping
+   * @throws QueryExecutionException
+   */
+  private static Map<String, Dictionary> getDictionaryMap(List<String> dictionaryColumnIdList,
+      AbsoluteTableIdentifier absoluteTableIdentifier) throws QueryExecutionException {
+    // this for dictionary unique identifier
+    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
+        getDictionaryColumnUniqueIdentifierList(dictionaryColumnIdList,
+            absoluteTableIdentifier.getCarbonTableIdentifier());
+    CacheProvider cacheProvider = CacheProvider.getInstance();
+    Cache forwardDictionaryCache = cacheProvider
+        .createCache(CacheType.FORWARD_DICTIONARY, absoluteTableIdentifier.getStorePath());
+    List<Dictionary> columnDictionaryList = null;
+    try {
+      columnDictionaryList = forwardDictionaryCache.getAll(dictionaryColumnUniqueIdentifiers);
+    } catch (CarbonUtilException e) {
+      throw new QueryExecutionException(e);
+    }
+    Map<String, Dictionary> columnDictionaryMap = new HashMap<>(columnDictionaryList.size());
+    for (int i = 0; i < dictionaryColumnUniqueIdentifiers.size(); i++) {
+      // TODO: null check for column dictionary, if cache size is less it
+      // might return null here, in that case throw exception
+      columnDictionaryMap.put(dictionaryColumnIdList.get(i), columnDictionaryList.get(i));
+    }
+    return columnDictionaryMap;
+  }
+
+  /**
+   * Below method will be used to get the dictionary column unique identifier
+   *
+   * @param dictionaryColumnIdList dictionary
+   * @param carbonTableIdentifier
+   * @return
+   */
+  private static List<DictionaryColumnUniqueIdentifier> getDictionaryColumnUniqueIdentifierList(
+      List<String> dictionaryColumnIdList, CarbonTableIdentifier carbonTableIdentifier) {
+    CarbonTable carbonTable =
+        CarbonMetadata.getInstance().getCarbonTable(carbonTableIdentifier.getTableUniqueName());
+    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
+        new ArrayList<>(dictionaryColumnIdList.size());
+    for (String columnIdentifier : dictionaryColumnIdList) {
+      CarbonDimension dimension = CarbonMetadata.getInstance()
+          .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnIdentifier);
+      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
+          new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
+              dimension.getDataType());
+      dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
+    }
+    return dictionaryColumnUniqueIdentifiers;
+  }
+
+  /**
+   * Below method will used to get the method will be used to get the measure
+   * block indexes to be read from the file
+   *
+   * @param queryMeasures              query measure
+   * @param expressionMeasure          measure present in the expression
+   * @param ordinalToBlockIndexMapping measure ordinal to block mapping
+   * @return block indexes
+   */
+  public static int[] getMeasureBlockIndexes(List<QueryMeasure> queryMeasures,
+      List<CarbonMeasure> expressionMeasure, Map<Integer, Integer> ordinalToBlockIndexMapping) {
+    Set<Integer> measureBlockIndex = new HashSet<Integer>();
+    for (int i = 0; i < queryMeasures.size(); i++) {
+      measureBlockIndex
+          .add(ordinalToBlockIndexMapping.get(queryMeasures.get(i).getMeasure().getOrdinal()));
+    }
+    for (int i = 0; i < expressionMeasure.size(); i++) {
+      measureBlockIndex.add(ordinalToBlockIndexMapping.get(expressionMeasure.get(i).getOrdinal()));
+    }
+    return ArrayUtils.toPrimitive(measureBlockIndex.toArray(new Integer[measureBlockIndex.size()]));
+  }
+
+  /**
+   * Below method will be used to get the masked byte range for dimension
+   * which is present in order by
+   *
+   * @param orderByDimensions order by dimension
+   * @param generator         key generator
+   * @param maskedRanges      masked byte range for dimension
+   * @return range of masked byte for order by dimension
+   */
+  public static int[][] getMaskedByteRangeForSorting(List<QueryDimension> orderByDimensions,
+      KeyGenerator generator, int[] maskedRanges) {
+    int[][] dimensionCompareIndex = new int[orderByDimensions.size()][];
+    int index = 0;
+    for (int i = 0; i < dimensionCompareIndex.length; i++) {
+      Set<Integer> integers = new TreeSet<Integer>();
+      if (!orderByDimensions.get(i).getDimension().getEncoder().contains(Encoding.DICTIONARY)
+          || orderByDimensions.get(i).getDimension().numberOfChild() > 0) {
+        continue;
+      }
+      int[] range =
+          generator.getKeyByteOffsets(orderByDimensions.get(i).getDimension().getKeyOrdinal());
+      for (int j = range[0]; j <= range[1]; j++) {
+        integers.add(j);
+      }
+      dimensionCompareIndex[index] = new int[integers.size()];
+      int j = 0;
+      for (Iterator<Integer> iterator = integers.iterator(); iterator.hasNext(); ) {
+        Integer integer = (Integer) iterator.next();
+        dimensionCompareIndex[index][j++] = integer.intValue();
+      }
+      index++;
+    }
+    for (int i = 0; i < dimensionCompareIndex.length; i++) {
+      if (null == dimensionCompareIndex[i]) {
+        continue;
+      }
+      int[] range = dimensionCompareIndex[i];
+      if (null != range) {
+        for (int j = 0; j < range.length; j++) {
+          for (int k = 0; k < maskedRanges.length; k++) {
+            if (range[j] == maskedRanges[k]) {
+              range[j] = k;
+              break;
+            }
+          }
+        }
+      }
+
+    }
+    return dimensionCompareIndex;
+  }
+
+  /**
+   * Below method will be used to get the masked key for sorting
+   *
+   * @param orderDimensions           query dimension
+   * @param generator                 key generator
+   * @param maskedByteRangeForSorting masked byte range for sorting
+   * @param maskedRanges              masked range
+   * @return masked byte range
+   * @throws QueryExecutionException
+   */
+  public static byte[][] getMaksedKeyForSorting(List<QueryDimension> orderDimensions,
+      KeyGenerator generator, int[][] maskedByteRangeForSorting, int[] maskedRanges)
+      throws QueryExecutionException {
+    byte[][] maskedKey = new byte[orderDimensions.size()][];
+    byte[] mdKey = null;
+    long[] key = null;
+    byte[] maskedMdKey = null;
+    try {
+      if (null != maskedByteRangeForSorting) {
+        for (int i = 0; i < maskedByteRangeForSorting.length; i++) {
+          if (null == maskedByteRangeForSorting[i]) {
+            continue;
+          }
+          key = new long[generator.getDimCount()];
+          maskedKey[i] = new byte[maskedByteRangeForSorting[i].length];
+          key[orderDimensions.get(i).getDimension().getKeyOrdinal()] = Long.MAX_VALUE;
+          mdKey = generator.generateKey(key);
+          maskedMdKey = new byte[maskedRanges.length];
+          for (int k = 0; k < maskedMdKey.length; k++) { // CHECKSTYLE:OFF
+            // Approval
+            // No:Approval-V1R2C10_001
+            maskedMdKey[k] = mdKey[maskedRanges[k]];
+          }
+          for (int j = 0; j < maskedByteRangeForSorting[i].length; j++) {
+            maskedKey[i][j] = maskedMdKey[maskedByteRangeForSorting[i][j]];
+          }// CHECKSTYLE:ON
+
+        }
+      }
+    } catch (KeyGenException e) {
+      throw new QueryExecutionException(e);
+    }
+    return maskedKey;
+  }
+
+  /**
+   * Below method will be used to get mapping whether dimension is present in
+   * order by or not
+   *
+   * @param sortedDimensions sort dimension present in order by query
+   * @param queryDimensions  query dimension
+   * @return sort dimension indexes
+   */
+  public static byte[] getSortDimensionIndexes(List<QueryDimension> sortedDimensions,
+      List<QueryDimension> queryDimensions) {
+    byte[] sortedDims = new byte[queryDimensions.size()];
+    int indexOf = 0;
+    for (int i = 0; i < sortedDims.length; i++) {
+      indexOf = sortedDimensions.indexOf(queryDimensions.get(i));
+      if (indexOf > -1) {
+        sortedDims[i] = 1;
+      }
+    }
+    return sortedDims;
+  }
+
+  /**
+   * Below method will be used to get the mapping of block index and its
+   * restructuring info
+   *
+   * @param queryDimensions   query dimension from query model
+   * @param segmentProperties segment properties
+   * @return map of block index to its restructuring info
+   * @throws KeyGenException if problem while key generation
+   */
+  public static Map<Integer, KeyStructureInfo> getColumnGroupKeyStructureInfo(
+      List<QueryDimension> queryDimensions, SegmentProperties segmentProperties)
+      throws KeyGenException {
+    Map<Integer, KeyStructureInfo> rowGroupToItsRSInfo = new HashMap<Integer, KeyStructureInfo>();
+    // get column group id and its ordinal mapping of column group
+    Map<Integer, List<Integer>> columnGroupAndItsOrdinalMappingForQuery =
+        getColumnGroupAndItsOrdinalMapping(queryDimensions);
+    KeyGenerator keyGenerator = segmentProperties.getDimensionKeyGenerator();
+
+    Iterator<Entry<Integer, List<Integer>>> iterator =
+        columnGroupAndItsOrdinalMappingForQuery.entrySet().iterator();
+    KeyStructureInfo restructureInfos = null;
+    while (iterator.hasNext()) {
+      Entry<Integer, List<Integer>> next = iterator.next();
+      restructureInfos = new KeyStructureInfo();
+      // sort the ordinal
+      List<Integer> ordinal = next.getValue();
+      Collections.sort(ordinal);
+      // get the masked byte range for column group
+      int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(ordinal, keyGenerator);
+      // max key for column group
+      byte[] maxKey = getMaxKeyBasedOnOrinal(ordinal, keyGenerator);
+      // get masked key for column group
+      int[] maksedByte = getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
+      restructureInfos.setKeyGenerator(keyGenerator);
+      restructureInfos.setMaskByteRanges(maskByteRanges);
+      restructureInfos.setMaxKey(maxKey);
+      restructureInfos.setMaskedBytes(maksedByte);
+      restructureInfos
+          .setBlockMdKeyStartOffset(getBlockMdKeyStartOffset(segmentProperties, ordinal));
+      rowGroupToItsRSInfo
+          .put(segmentProperties.getDimensionOrdinalToBlockMapping().get(ordinal.get(0)),
+              restructureInfos);
+    }
+    return rowGroupToItsRSInfo;
+  }
+
+  /**
+   * It return mdkey start index of given column group
+   * @param segmentProperties
+   * @param ordinal : column group ordinal
+   * @return
+   */
+  public static int getBlockMdKeyStartOffset(SegmentProperties segmentProperties,
+      List<Integer> ordinal) {
+    int[][] colGroups = segmentProperties.getColumnGroups();
+    int blockMdkeyStartOffset = 0;
+    for (int i = 0; i < colGroups.length; i++) {
+      if (QueryUtil.searchInArray(colGroups[i], ordinal.get(0))) {
+        break;
+      }
+      blockMdkeyStartOffset += segmentProperties.getDimensionColumnsValueSize()[i];
+    }
+    return blockMdkeyStartOffset;
+  }
+
+  /**
+   * return true if given key is found in array
+   *
+   * @param data
+   * @param key
+   * @return
+   */
+  public static boolean searchInArray(int[] data, int key) {
+    for (int i = 0; i < data.length; i++) {
+      if (key == data[i]) {
+        return true;
+      }
+    }
+    return false;
+  }
+
+  /**
+   * Below method will be used to create a mapping of column group columns
+   * this mapping will have column group id to all the dimension ordinal
+   * present in the column group This mapping will be used during query
+   * execution, to create a mask key for the column group dimension which will
+   * be used in aggregation and filter query as column group dimension will be
+   * stored in bit level
+   */
+  private static Map<Integer, List<Integer>> getColumnGroupAndItsOrdinalMapping(
+      List<QueryDimension> origdimensions) {
+
+    List<QueryDimension> dimensions = new ArrayList<QueryDimension>(origdimensions.size());
+    dimensions.addAll(origdimensions);
+    /**
+     * sort based on column group id
+     */
+    Collections.sort(dimensions, new Comparator<QueryDimension>() {
+
+      @Override public int compare(QueryDimension o1, QueryDimension o2) {
+        return Integer
+            .compare(o1.getDimension().columnGroupId(), o2.getDimension().columnGroupId());
+      }
+    });
+    // list of row groups this will store all the row group column
+    Map<Integer, List<Integer>> columnGroupAndItsOrdinalsMapping =
+        new HashMap<Integer, List<Integer>>();
+    // to store a column group
+    List<Integer> currentColumnGroup = null;
+    // current index
+    int index = 0;
+    // previous column group to check all the column of row id has bee
+    // selected
+    int prvColumnGroupId = -1;
+    while (index < dimensions.size()) {
+      // if dimension group id is not zero and it is same as the previous
+      // column group id
+      // then we need to add ordinal of that column as it belongs to same
+      // column group
+      if (!dimensions.get(index).getDimension().isColumnar()
+          && dimensions.get(index).getDimension().columnGroupId() == prvColumnGroupId) {
+        currentColumnGroup.add(dimensions.get(index).getDimension().getOrdinal());
+      }
+
+      // if dimension is not a columnar then it is column group column
+      else if (!dimensions.get(index).getDimension().isColumnar()) {
+        currentColumnGroup = new ArrayList<Integer>();
+        columnGroupAndItsOrdinalsMapping
+            .put(dimensions.get(index).getDimension().columnGroupId(), currentColumnGroup);
+        currentColumnGroup.add(dimensions.get(index).getDimension().getOrdinal());
+      }
+      // update the row id every time,this is required to group the
+      // columns
+      // of the same row group
+      prvColumnGroupId = dimensions.get(index).getDimension().columnGroupId();
+      index++;
+    }
+    return columnGroupAndItsOrdinalsMapping;
+  }
+
+  /**
+   * Below method will be used to get masked byte
+   *
+   * @param data           actual data
+   * @param maxKey         max key
+   * @param maskByteRanges mask byte range
+   * @param byteCount
+   * @return masked byte
+   */
+  public static byte[] getMaskedKey(byte[] data, byte[] maxKey, int[] maskByteRanges,
+      int byteCount) {
+    byte[] maskedKey = new byte[byteCount];
+    int counter = 0;
+    int byteRange = 0;
+    for (int i = 0; i < byteCount; i++) {
+      byteRange = maskByteRanges[i];
+      if (byteRange != -1) {
+        maskedKey[counter++] = (byte) (data[byteRange] & maxKey[byteRange]);
+      }
+    }
+    return maskedKey;
+  }
+
+  /**
+   * Below method will be used to fill block indexes of the query dimension
+   * which will be used in creating a output row Here is method we are passing
+   * two list which store the indexes one for dictionary column other for not
+   * dictionary column. This is done for specific purpose so that in one
+   * iteration we will be able to fill both type dimension block indexes
+   *
+   * @param queryDimensions                  dimension present in the query
+   * @param columnOrdinalToBlockIndexMapping column ordinal to block index mapping
+   * @param dictionaryDimensionBlockIndex    list to store dictionary column block indexes
+   * @param noDictionaryDimensionBlockIndex  list to store no dictionary block indexes
+   */
+  public static void fillQueryDimensionsBlockIndexes(List<QueryDimension> queryDimensions,
+      Map<Integer, Integer> columnOrdinalToBlockIndexMapping,
+      Set<Integer> dictionaryDimensionBlockIndex, List<Integer> noDictionaryDimensionBlockIndex) {
+    for (QueryDimension queryDimension : queryDimensions) {
+      if (CarbonUtil.hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)) {
+        dictionaryDimensionBlockIndex
+            .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal()));
+      } else {
+        noDictionaryDimensionBlockIndex
+            .add(columnOrdinalToBlockIndexMapping.get(queryDimension.getDimension().getOrdinal()));
+      }
+    }
+  }
+
+  /**
+   * Below method will be used to resolve the query model
+   * resolve will be setting the actual dimension and measure object
+   * as from driver only column name will be passes to avoid the heavy object
+   * serialization
+   *
+   * @param queryModel query model
+   */
+  public static void resolveQueryModel(QueryModel queryModel) {
+    CarbonMetadata.getInstance().addCarbonTable(queryModel.getTable());
+    // TODO need to load the table from table identifier
+    CarbonTable carbonTable = queryModel.getTable();
+    String tableName =
+        queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName();
+    // resolve query dimension
+    for (QueryDimension queryDimension : queryModel.getQueryDimension()) {
+      queryDimension
+          .setDimension(carbonTable.getDimensionByName(tableName, queryDimension.getColumnName()));
+    }
+    // resolve sort dimension
+    for (QueryDimension sortDimension : queryModel.getSortDimension()) {
+      sortDimension
+          .setDimension(carbonTable.getDimensionByName(tableName, sortDimension.getColumnName()));
+    }
+    // resolve query measure
+    for (QueryMeasure queryMeasure : queryModel.getQueryMeasures()) {
+      // in case of count start column name will  be count * so
+      // for count start add first measure if measure is not present
+      // than add first dimension as a measure
+      if (queryMeasure.getColumnName().equals("count(*)")) {
+        if (carbonTable.getMeasureByTableName(tableName).size() > 0) {
+          queryMeasure.setMeasure(carbonTable.getMeasureByTableName(tableName).get(0));
+        } else {
+          CarbonMeasure dummyMeasure = new CarbonMeasure(
+              carbonTable.getDimensionByTableName(tableName).get(0).getColumnSchema(), 0);
+          queryMeasure.setMeasure(dummyMeasure);
+        }
+      } else {
+        queryMeasure
+            .setMeasure(carbonTable.getMeasureByName(tableName, queryMeasure.getColumnName()));
+      }
+    }
+    //TODO need to handle expression
+  }
+
+  /**
+   * Below method will be used to get the index of number type aggregator
+   *
+   * @param aggType
+   * @return index in aggregator
+   */
+  public static int[] getNumberTypeIndex(List<String> aggType) {
+    List<Integer> indexList = new ArrayList<Integer>();
+    for (int i = 0; i < aggType.size(); i++) {
+      if (CarbonCommonConstants.SUM.equals(aggType.get(i)) || CarbonCommonConstants.AVERAGE
+          .equals(aggType.get(i))) {
+        indexList.add(i);
+      }
+    }
+    return ArrayUtils.toPrimitive(indexList.toArray(new Integer[indexList.size()]));
+  }
+
+  /**
+   * below method will be used to get the actual type aggregator
+   *
+   * @param aggType
+   * @return index in aggrgetor
+   */
+  public static int[] getActualTypeIndex(List<String> aggType) {
+    List<Integer> indexList = new ArrayList<Integer>();
+    for (int i = 0; i < aggType.size(); i++) {
+      if (!CarbonCommonConstants.SUM.equals(aggType.get(i)) && !CarbonCommonConstants.AVERAGE
+          .equals(aggType.get(i))) {
+        indexList.add(i);
+      }
+    }
+    return ArrayUtils.toPrimitive(indexList.toArray(new Integer[indexList.size()]));
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/executor/util/RestructureUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/util/RestructureUtil.java b/core/src/main/java/org/carbondata/scan/executor/util/RestructureUtil.java
new file mode 100644
index 0000000..92b469c
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/executor/util/RestructureUtil.java
@@ -0,0 +1,128 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.executor.util;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.scan.executor.infos.AggregatorInfo;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+
+/**
+ * Utility class for restructuring
+ */
+public class RestructureUtil {
+
+  /**
+   * Below method will be used to get the updated query dimension updation
+   * means, after restructuring some dimension will be not present in older
+   * table blocks in that case we need to select only those dimension out of
+   * query dimension which is present in the current table block
+   *
+   * @param queryDimensions
+   * @param tableBlockDimensions
+   * @return list of query dimension which is present in the table block
+   */
+  public static List<QueryDimension> getUpdatedQueryDimension(
+      List<QueryDimension> queryDimensions, List<CarbonDimension> tableBlockDimensions) {
+    List<QueryDimension> presentDimension =
+        new ArrayList<QueryDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    // selecting only those dimension which is present in the query
+    for (QueryDimension queryDimimension : queryDimensions) {
+      for (CarbonDimension tableDimension : tableBlockDimensions) {
+        if (tableDimension.equals(queryDimimension.getDimension())) {
+          presentDimension.add(queryDimimension);
+        }
+      }
+    }
+    return presentDimension;
+  }
+
+  /**
+   * Below method is to add dimension children for complex type dimension as
+   * internally we are creating dimension column for each each complex
+   * dimension so when complex query dimension request will come in the query,
+   * we need to add its children as it is hidden from the user For example if
+   * complex dimension is of Array of String[2] so we are storing 3 dimension
+   * and when user will query for complex type i.e. array type we need to add
+   * its children and then we will read respective block and create a tuple
+   * based on all three dimension
+   *
+   * @param queryDimensions      current query dimensions
+   * @param tableBlockDimensions dimensions which is present in the table block
+   * @return updated dimension(after adding complex type children)
+   */
+  public static List<CarbonDimension> addChildrenForComplexTypeDimension(
+      List<CarbonDimension> queryDimensions, List<CarbonDimension> tableBlockDimensions) {
+    List<CarbonDimension> updatedQueryDimension = new ArrayList<CarbonDimension>();
+    int numberOfChildren = 0;
+    for (CarbonDimension queryDimension : queryDimensions) {
+      // if number of child is zero, then it is not a complex dimension
+      // so directly add it query dimension
+      if (queryDimension.numberOfChild() == 0) {
+        updatedQueryDimension.add(queryDimension);
+      }
+      // if number of child is more than 1 then add all its children
+      numberOfChildren = queryDimension.getOrdinal() + queryDimension.numberOfChild();
+      for (int j = queryDimension.getOrdinal(); j < numberOfChildren; j++) {
+        updatedQueryDimension.add(tableBlockDimensions.get(j));
+      }
+    }
+    return updatedQueryDimension;
+  }
+
+  /**
+   * Below method will be used to get the aggregator info object
+   * in this method some of the properties which will be extracted
+   * from query measure and current block measures will be set
+   *
+   * @param queryMeasures        measures present in query
+   * @param currentBlockMeasures current block measures
+   * @return aggregator info
+   */
+  public static AggregatorInfo getAggregatorInfos(List<QueryMeasure> queryMeasures,
+      List<CarbonMeasure> currentBlockMeasures) {
+    AggregatorInfo aggregatorInfos = new AggregatorInfo();
+    int numberOfMeasureInQuery = queryMeasures.size();
+    int[] measureOrdinals = new int[numberOfMeasureInQuery];
+    Object[] defaultValues = new Object[numberOfMeasureInQuery];
+    boolean[] measureExistsInCurrentBlock = new boolean[numberOfMeasureInQuery];
+    int index = 0;
+    for (QueryMeasure queryMeasure : queryMeasures) {
+      measureOrdinals[index] = queryMeasure.getMeasure().getOrdinal();
+      // if query measure exists in current dimension measures
+      // then setting measure exists is true
+      // otherwise adding a default value of a measure
+      if (currentBlockMeasures.contains(queryMeasure.getMeasure())) {
+        measureExistsInCurrentBlock[index] = true;
+      } else {
+        defaultValues[index] = queryMeasure.getMeasure().getDefaultValue();
+      }
+      index++;
+    }
+    aggregatorInfos.setDefaultValues(defaultValues);
+    aggregatorInfos.setMeasureOrdinals(measureOrdinals);
+    aggregatorInfos.setMeasureExists(measureExistsInCurrentBlock);
+    return aggregatorInfos;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/BinaryExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/BinaryExpression.java b/core/src/main/java/org/carbondata/scan/expression/BinaryExpression.java
new file mode 100644
index 0000000..1ad334e
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/BinaryExpression.java
@@ -0,0 +1,59 @@
+/*
+ * 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.carbondata.scan.expression;
+
+public abstract class BinaryExpression extends Expression {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+  /**
+   *
+   */
+
+  protected Expression left;
+  protected Expression right;
+  protected boolean isRangeExpression;
+
+  public BinaryExpression(Expression left, Expression right) {
+    this.left = left;
+    this.right = right;
+    children.add(left);
+    children.add(right);
+  }
+
+  public Expression getLeft() {
+    return left;
+  }
+
+  public Expression getRight() {
+    return right;
+  }
+
+  public boolean isRangeExpression() {
+    return isRangeExpression;
+  }
+
+  public void setRangeExpression(boolean isRangeExpression) {
+    this.isRangeExpression = isRangeExpression;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java b/core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java
new file mode 100644
index 0000000..922e706
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java
@@ -0,0 +1,112 @@
+/*
+ * 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.carbondata.scan.expression;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class ColumnExpression extends LeafExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  private String columnName;
+
+  private boolean isDimension;
+
+  private int colIndex = -1;
+
+  private DataType dataType;
+
+  private CarbonDimension dimension;
+
+  private CarbonColumn carbonColumn;
+
+  public ColumnExpression(String columnName, DataType dataType) {
+    this.columnName = columnName;
+    this.dataType = dataType;
+
+  }
+
+  public CarbonDimension getDimension() {
+    return dimension;
+  }
+
+  public void setDimension(CarbonDimension dimension) {
+    this.dimension = dimension;
+  }
+
+  public String getColumnName() {
+    return columnName;
+  }
+
+  public void setColumnName(String columnName) {
+    this.columnName = columnName;
+  }
+
+  public boolean isDimension() {
+    return isDimension;
+  }
+
+  public void setDimension(boolean isDimension) {
+    this.isDimension = isDimension;
+  }
+
+  public int getColIndex() {
+    return colIndex;
+  }
+
+  public void setColIndex(int colIndex) {
+    this.colIndex = colIndex;
+  }
+
+  public DataType getDataType() {
+    return dataType;
+  }
+
+  public void setDataType(DataType dataType) {
+    this.dataType = dataType;
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) {
+    ExpressionResult expressionResult = new ExpressionResult(dataType, value.getVal(colIndex));
+    return expressionResult;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    // TODO Auto-generated method stub
+    return null;
+  }
+
+  @Override public String getString() {
+    // TODO Auto-generated method stub
+    return "ColumnExpression(" + columnName + ')';
+  }
+
+  public CarbonColumn getCarbonColumn() {
+    return carbonColumn;
+  }
+
+  public void setCarbonColumn(CarbonColumn carbonColumn) {
+    this.carbonColumn = carbonColumn;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/DataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/DataType.java b/core/src/main/java/org/carbondata/scan/expression/DataType.java
new file mode 100644
index 0000000..fc9d0bf
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/DataType.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.carbondata.scan.expression;
+
+public enum DataType {
+  StringType(0), DateType(1), TimestampType(2), BooleanType(1), IntegerType(3), FloatType(
+      4), LongType(5), DoubleType(6), NullType(7), DecimalType(8), ArrayType(9), StructType(10);
+  private int presedenceOrder;
+
+  private DataType(int value) {
+    this.presedenceOrder = value;
+  }
+
+  public int getPresedenceOrder() {
+    return presedenceOrder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/Expression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/Expression.java b/core/src/main/java/org/carbondata/scan/expression/Expression.java
new file mode 100644
index 0000000..01b4fee
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/Expression.java
@@ -0,0 +1,48 @@
+/*
+ * 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.carbondata.scan.expression;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public abstract class Expression implements Serializable {
+
+  private static final long serialVersionUID = -7568676723039530713L;
+  protected List<Expression> children =
+      new ArrayList<Expression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+  public abstract ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException;
+
+  public abstract ExpressionType getFilterExpressionType();
+
+  public List<Expression> getChildren() {
+    return children;
+  }
+
+  public abstract String getString();
+
+  // public abstract void  accept(ExpressionVisitor visitor);
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java b/core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
new file mode 100644
index 0000000..decbdc0
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
@@ -0,0 +1,413 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additiona   l 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.carbondata.scan.expression;
+
+import java.math.BigDecimal;
+import java.sql.Timestamp;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.ArrayList;
+import java.util.Date;
+import java.util.List;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonProperties;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+
+public class ExpressionResult implements Comparable<ExpressionResult> {
+
+  private static final long serialVersionUID = 1L;
+  protected DataType dataType;
+
+  protected Object value;
+
+  private List<ExpressionResult> expressionResults;
+
+  public ExpressionResult(DataType dataType, Object value) {
+    this.dataType = dataType;
+    this.value = value;
+  }
+
+  public ExpressionResult(List<ExpressionResult> expressionResults) {
+    this.expressionResults = expressionResults;
+  }
+
+  public void set(DataType dataType, Object value) {
+    this.dataType = dataType;
+    this.value = value;
+    this.expressionResults = null;
+  }
+
+  public DataType getDataType() {
+    return dataType;
+  }
+
+  //CHECKSTYLE:OFF Approval No:Approval-V1R2C10_009
+  public Integer getInt() throws FilterUnsupportedException {
+    if (value == null) {
+      return null;
+    }
+    try {
+      switch (this.getDataType()) {
+        case StringType:
+          try {
+            return Integer.parseInt(value.toString());
+          } catch (NumberFormatException e) {
+            throw new FilterUnsupportedException(e);
+          }
+
+        case IntegerType:
+        case DoubleType:
+
+          if (value instanceof Double) {
+            return ((Double) value).intValue();
+          }
+          return (Integer) value;
+
+        case TimestampType:
+
+          if (value instanceof Timestamp) {
+            return (int) (((Timestamp) value).getTime() % 1000);
+          } else {
+            return (Integer) value;
+          }
+
+        default:
+          throw new FilterUnsupportedException(
+              "Cannot convert" + this.getDataType().name() + " to integer type value");
+      }
+
+    } catch (ClassCastException e) {
+      throw new FilterUnsupportedException(
+          "Cannot convert" + this.getDataType().name() + " to Integer type value");
+    }
+  }
+
+  public String getString() {
+    if (value == null) {
+      return null;
+    }
+    switch (this.getDataType()) {
+      case TimestampType:
+        SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
+            .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+        if (value instanceof Timestamp) {
+          return parser.format((Timestamp) value);
+        } else {
+          return parser.format(new Timestamp((long) value / 1000));
+        }
+
+      default:
+        return value.toString();
+    }
+  }
+
+  public Double getDouble() throws FilterUnsupportedException {
+    if (value == null) {
+      return null;
+    }
+    try {
+      switch (this.getDataType()) {
+        case StringType:
+          try {
+            return Double.parseDouble(value.toString());
+          } catch (NumberFormatException e) {
+            throw new FilterUnsupportedException(e);
+          }
+
+        case IntegerType:
+          return ((Integer) value).doubleValue();
+        case LongType:
+          return ((Long) value).doubleValue();
+        case DoubleType:
+          return (Double) value;
+        case TimestampType:
+          if (value instanceof Timestamp) {
+            return (double) ((Timestamp) value).getTime() * 1000;
+          } else {
+            return (Double) (value);
+          }
+        default:
+          throw new FilterUnsupportedException(
+              "Cannot convert" + this.getDataType().name() + " to double type value");
+      }
+    } catch (ClassCastException e) {
+      throw new FilterUnsupportedException(
+          "Cannot convert" + this.getDataType().name() + " to Double type value");
+    }
+  }
+  //CHECKSTYLE:ON
+
+  public Long getLong() throws FilterUnsupportedException {
+    if (value == null) {
+      return null;
+    }
+    try {
+      switch (this.getDataType()) {
+        case StringType:
+          try {
+            return Long.parseLong(value.toString());
+          } catch (NumberFormatException e) {
+            throw new FilterUnsupportedException(e);
+          }
+
+        case IntegerType:
+          return (Long) value;
+        case LongType:
+          return (Long) value;
+        case DoubleType:
+          return (Long) value;
+        case TimestampType:
+          if (value instanceof Timestamp) {
+            return 1000 * ((Timestamp) value).getTime();
+          } else {
+            return (Long) value;
+          }
+        default:
+          throw new FilterUnsupportedException(
+              "Cannot convert" + this.getDataType().name() + " to Long type value");
+      }
+    } catch (ClassCastException e) {
+      throw new FilterUnsupportedException(
+          "Cannot convert" + this.getDataType().name() + " to Long type value");
+    }
+
+  }
+
+  //Add to judge for BigDecimal
+  public BigDecimal getDecimal() throws FilterUnsupportedException {
+    if (value == null) {
+      return null;
+    }
+    try {
+      switch (this.getDataType()) {
+        case StringType:
+          try {
+            return new BigDecimal(value.toString());
+          } catch (NumberFormatException e) {
+            throw new FilterUnsupportedException(e);
+          }
+
+        case IntegerType:
+          return new BigDecimal((int) value);
+        case LongType:
+          return new BigDecimal((long) value);
+        case DoubleType:
+          return new BigDecimal((double) value);
+        case DecimalType:
+          return new BigDecimal(value.toString());
+        case TimestampType:
+          if (value instanceof Timestamp) {
+            return new BigDecimal(1000 * ((Timestamp) value).getTime());
+          } else {
+            return new BigDecimal((long) value);
+          }
+        default:
+          throw new FilterUnsupportedException(
+              "Cannot convert" + this.getDataType().name() + " to Long type value");
+      }
+    } catch (ClassCastException e) {
+      throw new FilterUnsupportedException(
+          "Cannot convert" + this.getDataType().name() + " to Long type value");
+    }
+
+  }
+
+  public Long getTime() throws FilterUnsupportedException {
+    if (value == null) {
+      return null;
+    }
+    try {
+      switch (this.getDataType()) {
+        case StringType:
+          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+          Date dateToStr;
+          try {
+            dateToStr = parser.parse(value.toString());
+            return dateToStr.getTime() * 1000;
+          } catch (ParseException e) {
+            throw new FilterUnsupportedException(
+                "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
+          }
+        case IntegerType:
+        case LongType:
+          return (Long) value;
+        case DoubleType:
+          return (Long) value;
+        case TimestampType:
+          if (value instanceof Timestamp) {
+            return ((Timestamp) value).getTime() * 1000;
+          } else {
+            return (Long) value;
+          }
+        default:
+          throw new FilterUnsupportedException(
+              "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
+      }
+    } catch (ClassCastException e) {
+      throw new FilterUnsupportedException(
+          "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
+    }
+
+  }
+
+  public Boolean getBoolean() throws FilterUnsupportedException {
+    if (value == null) {
+      return null;
+    }
+    try {
+      switch (this.getDataType()) {
+        case StringType:
+          try {
+            return Boolean.parseBoolean(value.toString());
+          } catch (NumberFormatException e) {
+            throw new FilterUnsupportedException(e);
+          }
+
+        case BooleanType:
+          return Boolean.parseBoolean(value.toString());
+
+        default:
+          throw new FilterUnsupportedException(
+              "Cannot convert" + this.getDataType().name() + " to boolean type value");
+      }
+    } catch (ClassCastException e) {
+      throw new FilterUnsupportedException(
+          "Cannot convert" + this.getDataType().name() + " to Boolean type value");
+    }
+  }
+
+  public List<ExpressionResult> getList() {
+    if (null == expressionResults) {
+      List<ExpressionResult> a = new ArrayList<ExpressionResult>(20);
+      a.add(new ExpressionResult(dataType, value));
+      return a;
+    } else {
+      return expressionResults;
+    }
+  }
+
+  public List<String> getListAsString() {
+    List<String> evaluateResultListFinal = new ArrayList<String>(20);
+    List<ExpressionResult> evaluateResultList = getList();
+    for (ExpressionResult result : evaluateResultList) {
+      if (result.getString() == null) {
+        evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
+        continue;
+      }
+      evaluateResultListFinal.add(result.getString());
+    }
+    return evaluateResultListFinal;
+  }
+
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    if (null != expressionResults) {
+      result = prime * result + expressionResults.hashCode();
+    } else if (null != value) {
+      result = prime * result + value.toString().hashCode();
+    } else {
+      result = prime * result + "".hashCode();
+    }
+
+    return result;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (!(obj instanceof ExpressionResult)) {
+      return false;
+    }
+    if (this == obj) {
+      return true;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    ExpressionResult objToCompare = (ExpressionResult) obj;
+    boolean result = false;
+    if (this.value == objToCompare.value) {
+      return true;
+    }
+    try {
+      switch (this.getDataType()) {
+        case StringType:
+          result = this.getString().equals(objToCompare.getString());
+          break;
+        case IntegerType:
+          result = this.getInt().equals(objToCompare.getInt());
+          break;
+
+        case DoubleType:
+          result = this.getDouble().equals(objToCompare.getDouble());
+          break;
+        case TimestampType:
+          result = this.getLong().equals(objToCompare.getLong());
+          break;
+        default:
+          break;
+      }
+    } catch (FilterUnsupportedException ex) {
+      return false;
+    }
+
+    return result;
+  }
+
+  public boolean isNull() {
+    return value == null;
+  }
+
+  @Override public int compareTo(ExpressionResult o) {
+    try {
+      switch (o.dataType) {
+        case IntegerType:
+        case LongType:
+        case DoubleType:
+
+          Double d1 = this.getDouble();
+          Double d2 = o.getDouble();
+          return d1.compareTo(d2);
+        case DecimalType:
+          java.math.BigDecimal val1 = this.getDecimal();
+          java.math.BigDecimal val2 = o.getDecimal();
+          return val1.compareTo(val2);
+        case TimestampType:
+          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+          Date date1 = null;
+          Date date2 = null;
+          date1 = parser.parse(this.getString());
+          date2 = parser.parse(o.getString());
+          return date1.compareTo(date2);
+        case StringType:
+        default:
+          return this.getString().compareTo(o.getString());
+      }
+    } catch (Exception e) {
+      return -1;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/LeafExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/LeafExpression.java b/core/src/main/java/org/carbondata/scan/expression/LeafExpression.java
new file mode 100644
index 0000000..2392910
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/LeafExpression.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.carbondata.scan.expression;
+
+public abstract class LeafExpression extends Expression {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/LiteralExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/LiteralExpression.java b/core/src/main/java/org/carbondata/scan/expression/LiteralExpression.java
new file mode 100644
index 0000000..edbaf44
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/LiteralExpression.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.carbondata.scan.expression;
+
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class LiteralExpression extends LeafExpression {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+  private Object value;
+  private DataType dataType;
+
+  public LiteralExpression(Object value, DataType dataType) {
+    this.value = value;
+    this.dataType = dataType;
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) {
+    ExpressionResult expressionResult = new ExpressionResult(dataType, this.value);
+    return expressionResult;
+  }
+
+  public ExpressionResult getExpressionResult() {
+    ExpressionResult expressionResult = new ExpressionResult(dataType, this.value);
+    return expressionResult;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    // TODO Auto-generated method stub
+    return ExpressionType.LITERAL;
+  }
+
+  @Override public String getString() {
+    // TODO Auto-generated method stub
+    return "LiteralExpression(" + value + ')';
+  }
+
+  /**
+   * getLiteralExpDataType.
+   *
+   * @return
+   */
+  public DataType getLiteralExpDataType() {
+    return dataType;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/UnaryExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/UnaryExpression.java b/core/src/main/java/org/carbondata/scan/expression/UnaryExpression.java
new file mode 100644
index 0000000..0449b66
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/UnaryExpression.java
@@ -0,0 +1,33 @@
+/*
+ * 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.carbondata.scan.expression;
+
+public abstract class UnaryExpression extends Expression {
+
+  private static final long serialVersionUID = 1L;
+  protected Expression child;
+
+  public UnaryExpression(Expression child) {
+    this.child = child;
+    children.add(child);
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/UnknownExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/UnknownExpression.java b/core/src/main/java/org/carbondata/scan/expression/UnknownExpression.java
new file mode 100644
index 0000000..3f24e1c
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/UnknownExpression.java
@@ -0,0 +1,9 @@
+package org.carbondata.scan.expression;
+
+import java.util.List;
+
+public abstract class UnknownExpression extends Expression {
+
+  public abstract List<ColumnExpression> getAllColumnList();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java b/core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
new file mode 100644
index 0000000..e7ce159
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
@@ -0,0 +1,81 @@
+/*
+ * 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.carbondata.scan.expression.arithmetic;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class AddExpression extends BinaryArithmeticExpression {
+  private static final long serialVersionUID = 7999436055420911612L;
+
+  public AddExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult addExprLeftRes = left.evaluate(value);
+    ExpressionResult addExprRightRes = right.evaluate(value);
+    ExpressionResult val1 = addExprLeftRes;
+    ExpressionResult val2 = addExprRightRes;
+    if (addExprLeftRes.isNull() || addExprRightRes.isNull()) {
+      addExprLeftRes.set(addExprLeftRes.getDataType(), null);
+      return addExprLeftRes;
+    }
+
+    if (addExprLeftRes.getDataType() != addExprRightRes.getDataType()) {
+      if (addExprLeftRes.getDataType().getPresedenceOrder() < addExprRightRes.getDataType()
+          .getPresedenceOrder()) {
+        val2 = addExprLeftRes;
+        val1 = addExprRightRes;
+      }
+    }
+    switch (val1.getDataType()) {
+      case StringType:
+      case DoubleType:
+        addExprRightRes.set(DataType.DoubleType, val1.getDouble() + val2.getDouble());
+        break;
+      case IntegerType:
+        addExprRightRes.set(DataType.IntegerType, val1.getInt() + val2.getInt());
+        break;
+      case LongType:
+        addExprRightRes.set(DataType.LongType, val1.getLong() + val2.getLong());
+        break;
+      case DecimalType:
+        addExprRightRes.set(DataType.DecimalType, val1.getDecimal().add(val2.getDecimal()));
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "Incompatible datatype for applying Add Expression Filter " + val1.getDataType());
+    }
+    return addExprRightRes;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.ADD;
+  }
+
+  @Override public String getString() {
+    return "Add(" + left.getString() + ',' + right.getString() + ',';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/arithmetic/BinaryArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/arithmetic/BinaryArithmeticExpression.java b/core/src/main/java/org/carbondata/scan/expression/arithmetic/BinaryArithmeticExpression.java
new file mode 100644
index 0000000..9c109f7
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/BinaryArithmeticExpression.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.carbondata.scan.expression.arithmetic;
+
+import org.carbondata.scan.expression.BinaryExpression;
+import org.carbondata.scan.expression.Expression;
+
+public abstract class BinaryArithmeticExpression extends BinaryExpression {
+
+  private static final long serialVersionUID = 1L;
+
+  public BinaryArithmeticExpression(Expression left, Expression right) {
+    super(left, right);
+    // TODO Auto-generated constructor stub
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java b/core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
new file mode 100644
index 0000000..f4df604
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
@@ -0,0 +1,81 @@
+/*
+ * 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.carbondata.scan.expression.arithmetic;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class DivideExpression extends BinaryArithmeticExpression {
+  private static final long serialVersionUID = -7269266926782365612L;
+
+  public DivideExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult divideExprLeftRes = left.evaluate(value);
+    ExpressionResult divideExprRightRes = right.evaluate(value);
+    ExpressionResult val1 = divideExprLeftRes;
+    ExpressionResult val2 = divideExprRightRes;
+    if (divideExprLeftRes.isNull() || divideExprRightRes.isNull()) {
+      divideExprLeftRes.set(divideExprLeftRes.getDataType(), null);
+      return divideExprLeftRes;
+    }
+    if (divideExprLeftRes.getDataType() != divideExprRightRes.getDataType()) {
+      if (divideExprLeftRes.getDataType().getPresedenceOrder() < divideExprRightRes.getDataType()
+          .getPresedenceOrder()) {
+        val2 = divideExprLeftRes;
+        val1 = divideExprRightRes;
+      }
+    }
+    switch (val1.getDataType()) {
+      case StringType:
+      case DoubleType:
+        divideExprRightRes.set(DataType.DoubleType, val1.getDouble() / val2.getDouble());
+        break;
+      case IntegerType:
+        divideExprRightRes.set(DataType.IntegerType, val1.getInt() / val2.getInt());
+        break;
+      case LongType:
+        divideExprRightRes.set(DataType.LongType, val1.getLong() / val2.getLong());
+        break;
+      case DecimalType:
+        divideExprRightRes.set(DataType.DecimalType, val1.getDecimal().divide(val2.getDecimal()));
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "Incompatible datatype for applying Add Expression Filter " + divideExprLeftRes
+                .getDataType());
+    }
+    return divideExprRightRes;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.DIVIDE;
+  }
+
+  @Override public String getString() {
+    return "Divide(" + left.getString() + ',' + right.getString() + ')';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java b/core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
new file mode 100644
index 0000000..3347250
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.arithmetic;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class MultiplyExpression extends BinaryArithmeticExpression {
+  private static final long serialVersionUID = 1L;
+
+  public MultiplyExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult multiplyExprLeftRes = left.evaluate(value);
+    ExpressionResult multiplyExprRightRes = right.evaluate(value);
+    ExpressionResult val1 = multiplyExprLeftRes;
+    ExpressionResult val2 = multiplyExprRightRes;
+    if (multiplyExprLeftRes.isNull() || multiplyExprRightRes.isNull()) {
+      multiplyExprLeftRes.set(multiplyExprLeftRes.getDataType(), null);
+      return multiplyExprLeftRes;
+    }
+
+    if (multiplyExprLeftRes.getDataType() != multiplyExprRightRes.getDataType()) {
+      if (multiplyExprLeftRes.getDataType().getPresedenceOrder() < multiplyExprRightRes
+          .getDataType().getPresedenceOrder()) {
+        val2 = multiplyExprLeftRes;
+        val1 = multiplyExprRightRes;
+      }
+    }
+    switch (val1.getDataType()) {
+      case StringType:
+      case DoubleType:
+        multiplyExprRightRes.set(DataType.DoubleType, val1.getDouble() * val2.getDouble());
+        break;
+      case IntegerType:
+        multiplyExprRightRes.set(DataType.IntegerType, val1.getInt() * val2.getInt());
+        break;
+      case LongType:
+        multiplyExprRightRes.set(DataType.LongType, val1.getLong() * val2.getLong());
+        break;
+      case DecimalType:
+        multiplyExprRightRes
+            .set(DataType.DecimalType, val1.getDecimal().multiply(val2.getDecimal()));
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "Incompatible datatype for applying Add Expression Filter " + multiplyExprLeftRes
+                .getDataType());
+    }
+    return multiplyExprRightRes;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.MULTIPLY;
+  }
+
+  @Override public String getString() {
+    return "Substract(" + left.getString() + ',' + right.getString() + ')';
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java b/core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
new file mode 100644
index 0000000..35fec1c
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
@@ -0,0 +1,83 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.scan.expression.arithmetic;
+
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.ExpressionResult;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.intf.ExpressionType;
+import org.carbondata.scan.filter.intf.RowIntf;
+
+public class SubstractExpression extends BinaryArithmeticExpression {
+
+  private static final long serialVersionUID = -8304726440185363102L;
+
+  public SubstractExpression(Expression left, Expression right) {
+    super(left, right);
+  }
+
+  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
+    ExpressionResult subtractExprLeftRes = left.evaluate(value);
+    ExpressionResult subtractExprRightRes = right.evaluate(value);
+    ExpressionResult val1 = subtractExprLeftRes;
+    ExpressionResult val2 = subtractExprRightRes;
+    if (subtractExprLeftRes.isNull() || subtractExprRightRes.isNull()) {
+      subtractExprLeftRes.set(subtractExprLeftRes.getDataType(), null);
+      return subtractExprLeftRes;
+    }
+    if (subtractExprLeftRes.getDataType() != subtractExprRightRes.getDataType()) {
+      if (subtractExprLeftRes.getDataType().getPresedenceOrder() < subtractExprRightRes
+          .getDataType().getPresedenceOrder()) {
+        val2 = subtractExprLeftRes;
+        val1 = subtractExprRightRes;
+      }
+    }
+    switch (val1.getDataType()) {
+      case StringType:
+      case DoubleType:
+        subtractExprRightRes.set(DataType.DoubleType, val1.getDouble() - val2.getDouble());
+        break;
+      case IntegerType:
+        subtractExprRightRes.set(DataType.IntegerType, val1.getInt() - val2.getInt());
+        break;
+      case LongType:
+        subtractExprRightRes.set(DataType.LongType, val1.getLong() - val2.getLong());
+        break;
+      case DecimalType:
+        subtractExprRightRes
+            .set(DataType.DecimalType, val1.getDecimal().subtract(val2.getDecimal()));
+        break;
+      default:
+        throw new FilterUnsupportedException(
+            "Incompatible datatype for applying Add Expression Filter " + subtractExprLeftRes
+                .getDataType());
+    }
+    return subtractExprRightRes;
+  }
+
+  @Override public ExpressionType getFilterExpressionType() {
+    return ExpressionType.SUBSTRACT;
+  }
+
+  @Override public String getString() {
+    return "Substract(" + left.getString() + ',' + right.getString() + ')';
+  }
+}



[40/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
Refactor org.carbondata.query package (#692)

* fix style

* move query.carbon package one level up

* rename org.carbondata.query to org.carbondata.scan

* refactory


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

Branch: refs/heads/master
Commit: 1c725f5ba2e95e82e4922e7a8e0aa81cdc671a80
Parents: 2936050
Author: Jacky Li <ja...@huawei.com>
Authored: Sun Jun 19 02:30:21 2016 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sun Jun 19 00:00:21 2016 +0530

----------------------------------------------------------------------
 .../chunk/DimensionColumnDataChunk.java         |    2 +-
 .../impl/ColumnGroupDimensionDataChunk.java     |    2 +-
 .../impl/FixedLengthDimensionDataChunk.java     |    2 +-
 .../impl/VariableLengthDimensionDataChunk.java  |    2 +-
 .../org/carbondata/core/util/CarbonUtil.java    |    1 -
 .../core/util/DataFileFooterConverter.java      |  416 ++++++
 .../collector/ScannedResultCollector.java       |   43 -
 .../impl/ListBasedResultCollector.java          |  212 ---
 .../query/carbon/executor/QueryExecutor.java    |   40 -
 .../carbon/executor/QueryExecutorFactory.java   |   38 -
 .../exception/QueryExecutionException.java      |   96 --
 .../executor/impl/AbstractQueryExecutor.java    |  403 ------
 .../executor/impl/DetailQueryExecutor.java      |   46 -
 .../impl/DetailRawRecordQueryExecutor.java      |   24 -
 .../executor/impl/QueryExecutorProperties.java  |   78 --
 .../carbon/executor/infos/AggregatorInfo.java   |  149 ---
 .../executor/infos/BlockExecutionInfo.java      |  611 ---------
 .../carbon/executor/infos/KeyStructureInfo.java |  125 --
 .../query/carbon/executor/infos/SortInfo.java   |  125 --
 .../internal/InternalQueryExecutor.java         |   47 -
 .../query/carbon/executor/util/QueryUtil.java   |  718 ----------
 .../carbon/executor/util/RestructureUtil.java   |  128 --
 .../query/carbon/model/CarbonQueryPlan.java     |  243 ----
 .../carbon/model/DimensionAggregatorInfo.java   |  113 --
 .../query/carbon/model/QueryColumn.java         |  109 --
 .../query/carbon/model/QueryDimension.java      |   57 -
 .../query/carbon/model/QueryMeasure.java        |   60 -
 .../query/carbon/model/QueryModel.java          |  516 --------
 .../query/carbon/model/QuerySchemaInfo.java     |   86 --
 .../query/carbon/model/SortOrderType.java       |   57 -
 .../processor/AbstractDataBlockIterator.java    |  126 --
 .../carbon/processor/BlockletIterator.java      |   88 --
 .../carbon/processor/BlocksChunkHolder.java     |  125 --
 .../processor/impl/DataBlockIteratorImpl.java   |   56 -
 .../carbon/result/AbstractScannedResult.java    |  347 -----
 .../query/carbon/result/BatchRawResult.java     |   43 -
 .../query/carbon/result/BatchResult.java        |   86 --
 .../carbon/result/ListBasedResultWrapper.java   |   38 -
 .../carbondata/query/carbon/result/Result.java  |   70 -
 .../FixedLengthKeyResultComparator.java         |   71 -
 .../VariableLengthKeyResultComparator.java      |   80 --
 .../result/impl/FilterQueryScannedResult.java   |  128 --
 .../carbon/result/impl/ListBasedResult.java     |  153 ---
 .../impl/NonFilterQueryScannedResult.java       |  109 --
 .../AbstractDetailQueryResultIterator.java      |  134 --
 .../ChunkBasedDetailResultIterator.java         |   75 --
 .../result/iterator/ChunkRowIterator.java       |   79 --
 .../iterator/DetailQueryResultIterator.java     |   84 --
 .../iterator/MemoryBasedResultIterator.java     |   51 -
 .../result/iterator/RawResultIterator.java      |  167 ---
 .../preparator/QueryResultPreparator.java       |   10 -
 .../impl/AbstractQueryResultPreparator.java     |   87 --
 .../impl/DetailQueryResultPreparatorImpl.java   |  139 --
 .../impl/RawQueryResultPreparatorImpl.java      |  127 --
 .../carbon/scanner/AbstractBlockletScanner.java |   62 -
 .../query/carbon/scanner/BlockletScanner.java   |   41 -
 .../carbon/scanner/impl/FilterScanner.java      |  174 ---
 .../carbon/scanner/impl/NonFilterScanner.java   |   37 -
 .../query/carbon/util/DataTypeUtil.java         |  175 ---
 .../query/carbon/wrappers/ByteArrayWrapper.java |  202 ---
 .../carbonfilterinterface/ExpressionType.java   |   44 -
 .../FilterExecuterType.java                     |   28 -
 .../query/carbonfilterinterface/RowImpl.java    |   44 -
 .../query/carbonfilterinterface/RowIntf.java    |   31 -
 .../complex/querytypes/GenericQueryType.java    |   73 --
 .../evaluators/DimColumnExecuterFilterInfo.java |   32 -
 .../evaluators/FilterProcessorPlaceHolder.java  |   24 -
 .../query/expression/BinaryExpression.java      |   59 -
 .../query/expression/ColumnExpression.java      |  114 --
 .../carbondata/query/expression/DataType.java   |   34 -
 .../carbondata/query/expression/Expression.java |   51 -
 .../query/expression/ExpressionResult.java      |  413 ------
 .../query/expression/LeafExpression.java        |   24 -
 .../query/expression/LiteralExpression.java     |   68 -
 .../query/expression/UnaryExpression.java       |   33 -
 .../query/expression/UnknownExpression.java     |    9 -
 .../expression/arithmetic/AddExpression.java    |   81 --
 .../arithmetic/BinaryArithmeticExpression.java  |   34 -
 .../expression/arithmetic/DivideExpression.java |   81 --
 .../arithmetic/MultiplyExpression.java          |   83 --
 .../arithmetic/SubstractExpression.java         |   83 --
 .../BinaryConditionalExpression.java            |   37 -
 .../conditional/ConditionalExpression.java      |   43 -
 .../conditional/EqualToExpression.java          |   95 --
 .../GreaterThanEqualToExpression.java           |   85 --
 .../conditional/GreaterThanExpression.java      |   88 --
 .../expression/conditional/InExpression.java    |   99 --
 .../conditional/LessThanEqualToExpression.java  |   88 --
 .../conditional/LessThanExpression.java         |   90 --
 .../expression/conditional/ListExpression.java  |   57 -
 .../conditional/NotEqualsExpression.java        |   93 --
 .../expression/conditional/NotInExpression.java |  100 --
 .../exception/FilterUnsupportedException.java   |   93 --
 .../query/expression/logical/AndExpression.java |   63 -
 .../logical/BinaryLogicalExpression.java        |  127 --
 .../query/expression/logical/NotExpression.java |   58 -
 .../query/expression/logical/OrExpression.java  |   60 -
 .../filter/executer/AndFilterExecuterImpl.java  |   62 -
 .../executer/ColGroupFilterExecuterImpl.java    |  190 ---
 .../executer/ExcludeFilterExecuterImpl.java     |  206 ---
 .../query/filter/executer/FilterExecuter.java   |   45 -
 .../executer/IncludeFilterExecuterImpl.java     |  225 ----
 .../filter/executer/OrFilterExecuterImpl.java   |   52 -
 .../executer/RestructureFilterExecuterImpl.java |   55 -
 .../executer/RowLevelFilterExecuterImpl.java    |  331 -----
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |   66 -
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |   66 -
 ...velRangeLessThanEqualFilterExecuterImpl.java |   66 -
 .../RowLevelRangeLessThanFiterExecuterImpl.java |   65 -
 .../RowLevelRangeTypeExecuterFacory.java        |   90 --
 .../filter/resolver/AndFilterResolverImpl.java  |   51 -
 .../resolver/ConditionalFilterResolverImpl.java |  239 ----
 .../filter/resolver/FilterResolverIntf.java     |   98 --
 .../resolver/LogicalFilterResolverImpl.java     |  109 --
 .../resolver/RestructureFilterResolverImpl.java |  210 ---
 .../resolver/RowLevelFilterResolverImpl.java    |  143 --
 .../RowLevelRangeFilterResolverImpl.java        |  232 ----
 .../metadata/FilterResolverMetadata.java        |   62 -
 .../DimColumnResolvedFilterInfo.java            |  206 ---
 .../MeasureColumnResolvedFilterInfo.java        |  105 --
 .../visitable/ResolvedFilterInfoVisitable.java  |   38 -
 .../visitor/CustomTypeDictionaryVisitor.java    |   80 --
 .../visitor/DictionaryColumnVisitor.java        |   58 -
 .../visitor/FilterInfoTypeVisitorFactory.java   |   45 -
 .../visitor/NoDictionaryTypeVisitor.java        |   56 -
 .../visitor/ResolvedFilterInfoVisitorIntf.java  |   40 -
 .../filters/FilterExpressionProcessor.java      |  361 -----
 .../query/filters/FilterProcessor.java          |   60 -
 .../filters/measurefilter/util/FilterUtil.java  | 1234 ------------------
 .../schema/metadata/DimColumnFilterInfo.java    |   63 -
 .../query/util/DataFileFooterConverter.java     |  418 ------
 .../scan/collector/ScannedResultCollector.java  |   43 +
 .../impl/ListBasedResultCollector.java          |  212 +++
 .../carbondata/scan/executor/QueryExecutor.java |   40 +
 .../scan/executor/QueryExecutorFactory.java     |   38 +
 .../exception/QueryExecutionException.java      |   96 ++
 .../executor/impl/AbstractQueryExecutor.java    |  403 ++++++
 .../scan/executor/impl/DetailQueryExecutor.java |   46 +
 .../impl/DetailRawRecordQueryExecutor.java      |   24 +
 .../executor/impl/QueryExecutorProperties.java  |   78 ++
 .../scan/executor/infos/AggregatorInfo.java     |  149 +++
 .../scan/executor/infos/BlockExecutionInfo.java |  611 +++++++++
 .../scan/executor/infos/KeyStructureInfo.java   |  125 ++
 .../scan/executor/infos/SortInfo.java           |  125 ++
 .../scan/executor/util/QueryUtil.java           |  718 ++++++++++
 .../scan/executor/util/RestructureUtil.java     |  128 ++
 .../scan/expression/BinaryExpression.java       |   59 +
 .../scan/expression/ColumnExpression.java       |  112 ++
 .../carbondata/scan/expression/DataType.java    |   34 +
 .../carbondata/scan/expression/Expression.java  |   48 +
 .../scan/expression/ExpressionResult.java       |  413 ++++++
 .../scan/expression/LeafExpression.java         |   24 +
 .../scan/expression/LiteralExpression.java      |   68 +
 .../scan/expression/UnaryExpression.java        |   33 +
 .../scan/expression/UnknownExpression.java      |    9 +
 .../expression/arithmetic/AddExpression.java    |   81 ++
 .../arithmetic/BinaryArithmeticExpression.java  |   34 +
 .../expression/arithmetic/DivideExpression.java |   81 ++
 .../arithmetic/MultiplyExpression.java          |   83 ++
 .../arithmetic/SubstractExpression.java         |   83 ++
 .../BinaryConditionalExpression.java            |   37 +
 .../conditional/ConditionalExpression.java      |   43 +
 .../conditional/EqualToExpression.java          |   95 ++
 .../GreaterThanEqualToExpression.java           |   85 ++
 .../conditional/GreaterThanExpression.java      |   89 ++
 .../expression/conditional/InExpression.java    |   99 ++
 .../conditional/LessThanEqualToExpression.java  |   88 ++
 .../conditional/LessThanExpression.java         |   90 ++
 .../expression/conditional/ListExpression.java  |   57 +
 .../conditional/NotEqualsExpression.java        |   93 ++
 .../expression/conditional/NotInExpression.java |  100 ++
 .../exception/FilterUnsupportedException.java   |   93 ++
 .../scan/expression/logical/AndExpression.java  |   63 +
 .../logical/BinaryLogicalExpression.java        |  127 ++
 .../scan/expression/logical/NotExpression.java  |   58 +
 .../scan/expression/logical/OrExpression.java   |   60 +
 .../scan/filter/DimColumnFilterInfo.java        |   61 +
 .../scan/filter/FilterExpressionProcessor.java  |  360 +++++
 .../carbondata/scan/filter/FilterProcessor.java |   60 +
 .../org/carbondata/scan/filter/FilterUtil.java  | 1233 +++++++++++++++++
 .../scan/filter/GenericQueryType.java           |   73 ++
 .../filter/executer/AndFilterExecuterImpl.java  |   62 +
 .../executer/ColGroupFilterExecuterImpl.java    |  190 +++
 .../executer/DimColumnExecuterFilterInfo.java   |   32 +
 .../executer/ExcludeFilterExecuterImpl.java     |  205 +++
 .../scan/filter/executer/FilterExecuter.java    |   45 +
 .../executer/IncludeFilterExecuterImpl.java     |  224 ++++
 .../filter/executer/OrFilterExecuterImpl.java   |   52 +
 .../executer/RestructureFilterExecuterImpl.java |   55 +
 .../executer/RowLevelFilterExecuterImpl.java    |  331 +++++
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |   66 +
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |   66 +
 ...velRangeLessThanEqualFilterExecuterImpl.java |   66 +
 .../RowLevelRangeLessThanFiterExecuterImpl.java |   65 +
 .../RowLevelRangeTypeExecuterFacory.java        |   90 ++
 .../scan/filter/intf/ExpressionType.java        |   44 +
 .../scan/filter/intf/FilterExecuterType.java    |   28 +
 .../carbondata/scan/filter/intf/RowImpl.java    |   44 +
 .../carbondata/scan/filter/intf/RowIntf.java    |   31 +
 .../filter/resolver/AndFilterResolverImpl.java  |   51 +
 .../resolver/ConditionalFilterResolverImpl.java |  239 ++++
 .../filter/resolver/FilterResolverIntf.java     |   98 ++
 .../resolver/LogicalFilterResolverImpl.java     |  110 ++
 .../resolver/RestructureFilterResolverImpl.java |  211 +++
 .../resolver/RowLevelFilterResolverImpl.java    |  141 ++
 .../RowLevelRangeFilterResolverImpl.java        |  233 ++++
 .../metadata/FilterResolverMetadata.java        |   62 +
 .../DimColumnResolvedFilterInfo.java            |  206 +++
 .../MeasureColumnResolvedFilterInfo.java        |  105 ++
 .../visitable/ResolvedFilterInfoVisitable.java  |   38 +
 .../visitor/CustomTypeDictionaryVisitor.java    |   80 ++
 .../visitor/DictionaryColumnVisitor.java        |   58 +
 .../visitor/FilterInfoTypeVisitorFactory.java   |   45 +
 .../visitor/NoDictionaryTypeVisitor.java        |   56 +
 .../visitor/ResolvedFilterInfoVisitorIntf.java  |   40 +
 .../carbondata/scan/model/CarbonQueryPlan.java  |  243 ++++
 .../org/carbondata/scan/model/QueryColumn.java  |  109 ++
 .../carbondata/scan/model/QueryDimension.java   |   58 +
 .../org/carbondata/scan/model/QueryMeasure.java |   61 +
 .../org/carbondata/scan/model/QueryModel.java   |  516 ++++++++
 .../carbondata/scan/model/QuerySchemaInfo.java  |   86 ++
 .../carbondata/scan/model/SortOrderType.java    |   57 +
 .../processor/AbstractDataBlockIterator.java    |  126 ++
 .../scan/processor/BlockletIterator.java        |   88 ++
 .../scan/processor/BlocksChunkHolder.java       |  125 ++
 .../processor/impl/DataBlockIteratorImpl.java   |   56 +
 .../scan/result/AbstractScannedResult.java      |  347 +++++
 .../carbondata/scan/result/BatchRawResult.java  |   43 +
 .../org/carbondata/scan/result/BatchResult.java |   86 ++
 .../scan/result/ListBasedResultWrapper.java     |   57 +
 .../java/org/carbondata/scan/result/Result.java |   70 +
 .../result/impl/FilterQueryScannedResult.java   |  128 ++
 .../scan/result/impl/ListBasedResult.java       |  153 +++
 .../impl/NonFilterQueryScannedResult.java       |  109 ++
 .../AbstractDetailQueryResultIterator.java      |  128 ++
 .../scan/result/iterator/ChunkRowIterator.java  |   79 ++
 .../iterator/DetailQueryResultIterator.java     |   84 ++
 .../scan/result/iterator/RawResultIterator.java |  167 +++
 .../preparator/QueryResultPreparator.java       |   10 +
 .../impl/AbstractQueryResultPreparator.java     |   87 ++
 .../impl/DetailQueryResultPreparatorImpl.java   |  139 ++
 .../impl/RawQueryResultPreparatorImpl.java      |  127 ++
 .../scan/scanner/AbstractBlockletScanner.java   |   62 +
 .../scan/scanner/BlockletScanner.java           |   41 +
 .../scan/scanner/impl/FilterScanner.java        |  174 +++
 .../scan/scanner/impl/NonFilterScanner.java     |   37 +
 .../org/carbondata/scan/util/DataTypeUtil.java  |  175 +++
 .../scan/wrappers/ByteArrayWrapper.java         |  202 +++
 .../org/carbondata/query/QueryExecutor_UT.java  |   35 -
 .../carbon/executor/util/QueryUtilTest.java     |  133 --
 .../org/carbondata/scan/QueryExecutor_UT.java   |   35 +
 .../scan/executor/util/QueryUtilTest.java       |  133 ++
 dev/findbugs-exclude.xml                        |    4 +-
 .../carbondata/hadoop/CarbonInputFormat.java    |   16 +-
 .../carbondata/hadoop/CarbonRecordReader.java   |   10 +-
 .../hadoop/util/CarbonInputFormatUtil.java      |   14 +-
 .../hadoop/ft/CarbonInputFormat_FT.java         |   11 +-
 .../hadoop/ft/CarbonInputMapperTest.java        |   10 +-
 .../test/util/ObjectSerializationUtilTest.java  |   10 +-
 .../spark/merger/CarbonCompactionExecutor.java  |   16 +-
 .../spark/merger/RowResultMerger.java           |    4 +-
 .../spark/merger/TupleConversionAdapter.java    |    2 +-
 .../spark/partition/api/DataPartitioner.java    |    2 +-
 .../api/impl/QueryPartitionHelper.java          |    2 +-
 .../api/impl/SampleDataPartitionerImpl.java     |    2 +-
 .../readsupport/SparkRowReadSupportImpl.java    |    2 +-
 .../carbondata/spark/util/CarbonQueryUtil.java  |    2 +-
 .../apache/spark/sql/CarbonBoundReference.scala |    2 +-
 .../sql/CarbonDatasourceHadoopRelation.scala    |    2 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |    2 +-
 .../org/apache/spark/sql/CarbonOperators.scala  |    2 +-
 .../spark/sql/SparkUnknownExpression.scala      |    8 +-
 .../org/carbondata/spark/CarbonFilters.scala    |    6 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |    2 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   11 +-
 .../carbondata/spark/util/CarbonScalaUtil.scala |    2 +-
 276 files changed, 15220 insertions(+), 15680 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
index e3371ac..75c29d2 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/DimensionColumnDataChunk.java
@@ -18,7 +18,7 @@
  */
 package org.carbondata.core.carbon.datastore.chunk;
 
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
 
 /**
  * Interface for dimension column chunk.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
index 55c38d3..07d2e2e 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
@@ -20,7 +20,7 @@ package org.carbondata.core.carbon.datastore.chunk.impl;
 
 import org.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
 import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
 
 /**
  * This class is holder of the dimension column chunk data of the fixed length

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
index 3a544a4..08ae703 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/FixedLengthDimensionDataChunk.java
@@ -20,7 +20,7 @@ package org.carbondata.core.carbon.datastore.chunk.impl;
 
 import org.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
 import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
 
 /**
  * This class is holder of the dimension column chunk data of the fixed length

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
index 9ce3257..51306b5 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/VariableLengthDimensionDataChunk.java
@@ -4,7 +4,7 @@ import java.util.List;
 
 import org.carbondata.core.carbon.datastore.chunk.DimensionChunkAttributes;
 import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
+import org.carbondata.scan.executor.infos.KeyStructureInfo;
 
 /**
  * This class is holder of the dimension column chunk data of the variable

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 2772362..42628b2 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -71,7 +71,6 @@ import org.carbondata.core.metadata.SliceMetaData;
 import org.carbondata.core.metadata.ValueEncoderMeta;
 import org.carbondata.core.reader.CarbonFooterReader;
 import org.carbondata.core.vo.ColumnGroupModel;
-import org.carbondata.query.util.DataFileFooterConverter;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java b/core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
new file mode 100644
index 0000000..6165731
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
@@ -0,0 +1,416 @@
+/*
+ * 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.carbondata.core.util;
+
+import java.io.ByteArrayInputStream;
+import java.io.IOException;
+import java.io.ObjectInputStream;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.BitSet;
+import java.util.Iterator;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
+import org.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
+import org.carbondata.core.carbon.metadata.blocklet.SegmentInfo;
+import org.carbondata.core.carbon.metadata.blocklet.compressor.ChunkCompressorMeta;
+import org.carbondata.core.carbon.metadata.blocklet.compressor.CompressionCodec;
+import org.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
+import org.carbondata.core.carbon.metadata.blocklet.datachunk.PresenceMeta;
+import org.carbondata.core.carbon.metadata.blocklet.index.BlockletBTreeIndex;
+import org.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
+import org.carbondata.core.carbon.metadata.blocklet.index.BlockletMinMaxIndex;
+import org.carbondata.core.carbon.metadata.blocklet.sort.SortState;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+import org.carbondata.core.metadata.ValueEncoderMeta;
+import org.carbondata.core.reader.CarbonFooterReader;
+import org.carbondata.format.FileFooter;
+
+/**
+ * Below class will be used to convert the thrift object of data file
+ * meta data to wrapper object
+ */
+class DataFileFooterConverter {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DataFileFooterConverter.class.getName());
+
+  /**
+   * Below method will be used to convert thrift file meta to wrapper file meta
+   */
+  public DataFileFooter readDataFileFooter(String filePath, long blockOffset, long blockLength)
+      throws IOException {
+    DataFileFooter dataFileFooter = new DataFileFooter();
+    FileHolder fileReader = null;
+    try {
+      long completeBlockLength = blockOffset + blockLength;
+      long footerPointer = completeBlockLength - 8;
+      fileReader = FileFactory.getFileHolder(FileFactory.getFileType(filePath));
+      long actualFooterOffset = fileReader.readLong(filePath, footerPointer);
+      CarbonFooterReader reader = new CarbonFooterReader(filePath, actualFooterOffset);
+      FileFooter footer = reader.readFooter();
+      dataFileFooter.setVersionId(footer.getVersion());
+      dataFileFooter.setNumberOfRows(footer.getNum_rows());
+      dataFileFooter.setSegmentInfo(getSegmentInfo(footer.getSegment_info()));
+      List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
+      List<org.carbondata.format.ColumnSchema> table_columns = footer.getTable_columns();
+      for (int i = 0; i < table_columns.size(); i++) {
+        columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
+      }
+      dataFileFooter.setColumnInTable(columnSchemaList);
+
+      List<org.carbondata.format.BlockletIndex> leaf_node_indices_Thrift =
+          footer.getBlocklet_index_list();
+      List<BlockletIndex> blockletIndexList = new ArrayList<BlockletIndex>();
+      for (int i = 0; i < leaf_node_indices_Thrift.size(); i++) {
+        BlockletIndex blockletIndex = getBlockletIndex(leaf_node_indices_Thrift.get(i));
+        blockletIndexList.add(blockletIndex);
+      }
+
+      List<org.carbondata.format.BlockletInfo> leaf_node_infos_Thrift =
+          footer.getBlocklet_info_list();
+      List<BlockletInfo> blockletInfoList = new ArrayList<BlockletInfo>();
+      for (int i = 0; i < leaf_node_infos_Thrift.size(); i++) {
+        BlockletInfo blockletInfo = getBlockletInfo(leaf_node_infos_Thrift.get(i));
+        blockletInfo.setBlockletIndex(blockletIndexList.get(i));
+        blockletInfoList.add(blockletInfo);
+      }
+      dataFileFooter.setBlockletList(blockletInfoList);
+      dataFileFooter.setBlockletIndex(getBlockletIndexForDataFileFooter(blockletIndexList));
+    } finally {
+      if (null != fileReader) {
+        fileReader.finish();
+      }
+    }
+    return dataFileFooter;
+  }
+
+  /**
+   * Below method will be used to get blocklet index for data file meta
+   *
+   * @param blockletIndexList
+   * @return blocklet index
+   */
+  private BlockletIndex getBlockletIndexForDataFileFooter(List<BlockletIndex> blockletIndexList) {
+    BlockletIndex blockletIndex = new BlockletIndex();
+    BlockletBTreeIndex blockletBTreeIndex = new BlockletBTreeIndex();
+    blockletBTreeIndex.setStartKey(blockletIndexList.get(0).getBtreeIndex().getStartKey());
+    blockletBTreeIndex
+        .setEndKey(blockletIndexList.get(blockletIndexList.size() - 1).getBtreeIndex().getEndKey());
+    blockletIndex.setBtreeIndex(blockletBTreeIndex);
+    byte[][] currentMinValue = blockletIndexList.get(0).getMinMaxIndex().getMinValues().clone();
+    byte[][] currentMaxValue = blockletIndexList.get(0).getMinMaxIndex().getMaxValues().clone();
+    byte[][] minValue = null;
+    byte[][] maxValue = null;
+    for (int i = 1; i < blockletIndexList.size(); i++) {
+      minValue = blockletIndexList.get(i).getMinMaxIndex().getMinValues();
+      maxValue = blockletIndexList.get(i).getMinMaxIndex().getMaxValues();
+      for (int j = 0; j < maxValue.length; j++) {
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMinValue[j], minValue[j]) > 0) {
+          currentMinValue[j] = minValue[j].clone();
+        }
+        if (ByteUtil.UnsafeComparer.INSTANCE.compareTo(currentMaxValue[j], maxValue[j]) < 0) {
+          currentMaxValue[j] = maxValue[j].clone();
+        }
+      }
+    }
+
+    BlockletMinMaxIndex minMax = new BlockletMinMaxIndex();
+    minMax.setMaxValues(currentMaxValue);
+    minMax.setMinValues(currentMinValue);
+    blockletIndex.setMinMaxIndex(minMax);
+    return blockletIndex;
+  }
+
+  private ColumnSchema thriftColumnSchmeaToWrapperColumnSchema(
+      org.carbondata.format.ColumnSchema externalColumnSchema) {
+    ColumnSchema wrapperColumnSchema = new ColumnSchema();
+    wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
+    wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
+    wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
+    wrapperColumnSchema
+        .setDataType(thriftDataTyopeToWrapperDataType(externalColumnSchema.data_type));
+    wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
+    List<Encoding> encoders = new ArrayList<Encoding>();
+    for (org.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {
+      encoders.add(fromExternalToWrapperEncoding(encoder));
+    }
+    wrapperColumnSchema.setEncodingList(encoders);
+    wrapperColumnSchema.setNumberOfChild(externalColumnSchema.getNum_child());
+    wrapperColumnSchema.setPrecision(externalColumnSchema.getPrecision());
+    wrapperColumnSchema.setColumnGroup(externalColumnSchema.getColumn_group_id());
+    wrapperColumnSchema.setScale(externalColumnSchema.getScale());
+    wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
+    wrapperColumnSchema.setAggregateFunction(externalColumnSchema.getAggregate_function());
+    return wrapperColumnSchema;
+  }
+
+  /**
+   * Below method is to convert the blocklet info of the thrift to wrapper
+   * blocklet info
+   *
+   * @param blockletInfoThrift blocklet info of the thrift
+   * @return blocklet info wrapper
+   */
+  private BlockletInfo getBlockletInfo(org.carbondata.format.BlockletInfo blockletInfoThrift) {
+    BlockletInfo blockletInfo = new BlockletInfo();
+    List<DataChunk> dimensionColumnChunk = new ArrayList<DataChunk>();
+    List<DataChunk> measureChunk = new ArrayList<DataChunk>();
+    Iterator<org.carbondata.format.DataChunk> column_data_chunksIterator =
+        blockletInfoThrift.getColumn_data_chunksIterator();
+    while (column_data_chunksIterator.hasNext()) {
+      org.carbondata.format.DataChunk next = column_data_chunksIterator.next();
+      if (next.isRowMajor()) {
+        dimensionColumnChunk.add(getDataChunk(next, false));
+      } else if (next.getEncoders().contains(org.carbondata.format.Encoding.DELTA)) {
+        measureChunk.add(getDataChunk(next, true));
+      } else {
+
+        dimensionColumnChunk.add(getDataChunk(next, false));
+      }
+    }
+    blockletInfo.setDimensionColumnChunk(dimensionColumnChunk);
+    blockletInfo.setMeasureColumnChunk(measureChunk);
+    blockletInfo.setNumberOfRows(blockletInfoThrift.getNum_rows());
+    return blockletInfo;
+  }
+
+  /**
+   * Below method is convert the thrift encoding to wrapper encoding
+   *
+   * @param encoderThrift thrift encoding
+   * @return wrapper encoding
+   */
+  private Encoding fromExternalToWrapperEncoding(org.carbondata.format.Encoding encoderThrift) {
+    switch (encoderThrift) {
+      case DICTIONARY:
+        return Encoding.DICTIONARY;
+      case DELTA:
+        return Encoding.DELTA;
+      case RLE:
+        return Encoding.RLE;
+      case INVERTED_INDEX:
+        return Encoding.INVERTED_INDEX;
+      case BIT_PACKED:
+        return Encoding.BIT_PACKED;
+      case DIRECT_DICTIONARY:
+        return Encoding.DIRECT_DICTIONARY;
+      default:
+        return Encoding.DICTIONARY;
+    }
+  }
+
+  /**
+   * Below method will be used to convert the thrift compression to wrapper
+   * compression codec
+   *
+   * @param compressionCodecThrift
+   * @return wrapper compression codec
+   */
+  private CompressionCodec getCompressionCodec(
+      org.carbondata.format.CompressionCodec compressionCodecThrift) {
+    switch (compressionCodecThrift) {
+      case SNAPPY:
+        return CompressionCodec.SNAPPY;
+      default:
+        return CompressionCodec.SNAPPY;
+    }
+  }
+
+  /**
+   * Below method will be used to convert thrift segment object to wrapper
+   * segment object
+   *
+   * @param segmentInfo thrift segment info object
+   * @return wrapper segment info object
+   */
+  private SegmentInfo getSegmentInfo(org.carbondata.format.SegmentInfo segmentInfo) {
+    SegmentInfo info = new SegmentInfo();
+    int[] cardinality = new int[segmentInfo.getColumn_cardinalities().size()];
+    for (int i = 0; i < cardinality.length; i++) {
+      cardinality[i] = segmentInfo.getColumn_cardinalities().get(i);
+    }
+    info.setColumnCardinality(cardinality);
+    info.setNumberOfColumns(segmentInfo.getNum_cols());
+    return info;
+  }
+
+  /**
+   * Below method will be used to convert the blocklet index of thrift to
+   * wrapper
+   *
+   * @param blockletIndexThrift
+   * @return blocklet index wrapper
+   */
+  private BlockletIndex getBlockletIndex(org.carbondata.format.BlockletIndex blockletIndexThrift) {
+    org.carbondata.format.BlockletBTreeIndex btreeIndex = blockletIndexThrift.getB_tree_index();
+    org.carbondata.format.BlockletMinMaxIndex minMaxIndex = blockletIndexThrift.getMin_max_index();
+    return new BlockletIndex(
+        new BlockletBTreeIndex(btreeIndex.getStart_key(), btreeIndex.getEnd_key()),
+        new BlockletMinMaxIndex(minMaxIndex.getMin_values(), minMaxIndex.getMax_values()));
+  }
+
+  /**
+   * Below method will be used to convert the thrift compression meta to
+   * wrapper chunk compression meta
+   *
+   * @param chunkCompressionMetaThrift
+   * @return chunkCompressionMetaWrapper
+   */
+  private ChunkCompressorMeta getChunkCompressionMeta(
+      org.carbondata.format.ChunkCompressionMeta chunkCompressionMetaThrift) {
+    ChunkCompressorMeta compressorMeta = new ChunkCompressorMeta();
+    compressorMeta
+        .setCompressor(getCompressionCodec(chunkCompressionMetaThrift.getCompression_codec()));
+    compressorMeta.setCompressedSize(chunkCompressionMetaThrift.getTotal_compressed_size());
+    compressorMeta.setUncompressedSize(chunkCompressionMetaThrift.getTotal_uncompressed_size());
+    return compressorMeta;
+  }
+
+  /**
+   * Below method will be used to convert the thrift data type to wrapper data
+   * type
+   *
+   * @param dataTypeThrift
+   * @return dataType wrapper
+   */
+  private DataType thriftDataTyopeToWrapperDataType(org.carbondata.format.DataType dataTypeThrift) {
+    switch (dataTypeThrift) {
+      case STRING:
+        return DataType.STRING;
+      case INT:
+        return DataType.INT;
+      case LONG:
+        return DataType.LONG;
+      case DOUBLE:
+        return DataType.DOUBLE;
+      case DECIMAL:
+        return DataType.DECIMAL;
+      case TIMESTAMP:
+        return DataType.TIMESTAMP;
+      case ARRAY:
+        return DataType.ARRAY;
+      case STRUCT:
+        return DataType.STRUCT;
+      default:
+        return DataType.STRING;
+    }
+  }
+
+  /**
+   * Below method will be used to convert the thrift presence meta to wrapper
+   * presence meta
+   *
+   * @param presentMetadataThrift
+   * @return wrapper presence meta
+   */
+  private PresenceMeta getPresenceMeta(org.carbondata.format.PresenceMeta presentMetadataThrift) {
+    PresenceMeta presenceMeta = new PresenceMeta();
+    presenceMeta.setRepresentNullValues(presentMetadataThrift.isRepresents_presence());
+    presenceMeta.setBitSet(BitSet.valueOf(presentMetadataThrift.getPresent_bit_stream()));
+    return presenceMeta;
+  }
+
+  /**
+   * Below method will be used to convert the thrift object to wrapper object
+   *
+   * @param sortStateThrift
+   * @return wrapper sort state object
+   */
+  private SortState getSortState(org.carbondata.format.SortState sortStateThrift) {
+    if (sortStateThrift == org.carbondata.format.SortState.SORT_EXPLICIT) {
+      return SortState.SORT_EXPLICT;
+    } else if (sortStateThrift == org.carbondata.format.SortState.SORT_NATIVE) {
+      return SortState.SORT_NATIVE;
+    } else {
+      return SortState.SORT_NONE;
+    }
+  }
+
+  /**
+   * Below method will be used to convert the thrift data chunk to wrapper
+   * data chunk
+   *
+   * @param datachunkThrift
+   * @return wrapper data chunk
+   */
+  private DataChunk getDataChunk(org.carbondata.format.DataChunk datachunkThrift,
+      boolean isPresenceMetaPresent) {
+    DataChunk dataChunk = new DataChunk();
+    dataChunk.setColumnUniqueIdList(datachunkThrift.getColumn_ids());
+    dataChunk.setDataPageLength(datachunkThrift.getData_page_length());
+    dataChunk.setDataPageOffset(datachunkThrift.getData_page_offset());
+    if (isPresenceMetaPresent) {
+      dataChunk.setNullValueIndexForColumn(getPresenceMeta(datachunkThrift.getPresence()));
+    }
+    dataChunk.setRlePageLength(datachunkThrift.getRle_page_length());
+    dataChunk.setRlePageOffset(datachunkThrift.getRle_page_offset());
+    dataChunk.setRowMajor(datachunkThrift.isRowMajor());
+    dataChunk.setRowIdPageLength(datachunkThrift.getRowid_page_length());
+    dataChunk.setRowIdPageOffset(datachunkThrift.getRowid_page_offset());
+    dataChunk.setSortState(getSortState(datachunkThrift.getSort_state()));
+    dataChunk.setChunkCompressionMeta(getChunkCompressionMeta(datachunkThrift.getChunk_meta()));
+    List<Encoding> encodingList = new ArrayList<Encoding>(datachunkThrift.getEncoders().size());
+    for (int i = 0; i < datachunkThrift.getEncoders().size(); i++) {
+      encodingList.add(fromExternalToWrapperEncoding(datachunkThrift.getEncoders().get(i)));
+    }
+    dataChunk.setEncoderList(encodingList);
+    if (encodingList.contains(Encoding.DELTA)) {
+      List<ByteBuffer> thriftEncoderMeta = datachunkThrift.getEncoder_meta();
+      List<ValueEncoderMeta> encodeMetaList =
+          new ArrayList<ValueEncoderMeta>(thriftEncoderMeta.size());
+      for (int i = 0; i < thriftEncoderMeta.size(); i++) {
+        encodeMetaList.add(deserializeEncoderMeta(thriftEncoderMeta.get(i).array()));
+      }
+      dataChunk.setValueEncoderMeta(encodeMetaList);
+    }
+    return dataChunk;
+  }
+
+  /**
+   * Below method will be used to convert the encode metadata to
+   * ValueEncoderMeta object
+   *
+   * @param encoderMeta
+   * @return ValueEncoderMeta object
+   */
+  private ValueEncoderMeta deserializeEncoderMeta(byte[] encoderMeta) {
+    // TODO : should remove the unnecessary fields.
+    ByteArrayInputStream aos = null;
+    ObjectInputStream objStream = null;
+    ValueEncoderMeta meta = null;
+    try {
+      aos = new ByteArrayInputStream(encoderMeta);
+      objStream = new ObjectInputStream(aos);
+      meta = (ValueEncoderMeta) objStream.readObject();
+    } catch (ClassNotFoundException e) {
+      LOGGER.error(e);
+    } catch (IOException e) {
+      CarbonUtil.closeStreams(objStream);
+    }
+    return meta;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java b/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java
deleted file mode 100644
index 9e5d401..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/collector/ScannedResultCollector.java
+++ /dev/null
@@ -1,43 +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.carbondata.query.carbon.collector;
-
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Interface which will be used to aggregate the scan result
- */
-public interface ScannedResultCollector {
-
-  /**
-   * Below method will be used to aggregate the scanned result
-   *
-   * @param scannedResult scanned result
-   * @return how many records was aggregated
-   */
-  int collectData(AbstractScannedResult scannedResult, int batchSize);
-
-  /**
-   * Below method will be used to get the aggregated result
-   *
-   * @return
-   */
-  Result getCollectedResult();
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java b/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java
deleted file mode 100644
index 30d33b8..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/collector/impl/ListBasedResultCollector.java
+++ /dev/null
@@ -1,212 +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.carbondata.query.carbon.collector.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.carbon.collector.ScannedResultCollector;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.impl.ListBasedResult;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * It is not a collector it is just a scanned result holder.
- *
- */
-public class ListBasedResultCollector implements ScannedResultCollector {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(ListBasedResultCollector.class.getName());
-
-  /**
-   * to keep a track of number of row processed to handle limit push down in
-   * case of detail query scenario
-   */
-  private int rowCounter;
-
-  /**
-   * dimension values list
-   */
-  private List<ListBasedResultWrapper> listBasedResult;
-
-  /**
-   * restructuring info
-   */
-  private KeyStructureInfo restructureInfos;
-
-  /**
-   * table block execution infos
-   */
-  private BlockExecutionInfo tableBlockExecutionInfos;
-
-  private int[] measuresOrdinal;
-
-  /**
-   * to check whether measure exists in current table block or not this to
-   * handle restructuring scenario
-   */
-  private boolean[] isMeasureExistsInCurrentBlock;
-
-  /**
-   * default value of the measures in case of restructuring some measure wont
-   * be present in the table so in that default value will be used to
-   * aggregate the data for that measure columns
-   */
-  private Object[] measureDefaultValue;
-
-  /**
-   * measure datatypes.
-   */
-  private DataType[] measureDatatypes;
-
-  public ListBasedResultCollector(BlockExecutionInfo blockExecutionInfos) {
-    this.tableBlockExecutionInfos = blockExecutionInfos;
-    restructureInfos = blockExecutionInfos.getKeyStructureInfo();
-    measuresOrdinal = tableBlockExecutionInfos.getAggregatorInfo().getMeasureOrdinals();
-    isMeasureExistsInCurrentBlock = tableBlockExecutionInfos.getAggregatorInfo().getMeasureExists();
-    measureDefaultValue = tableBlockExecutionInfos.getAggregatorInfo().getDefaultValues();
-    this.measureDatatypes = tableBlockExecutionInfos.getAggregatorInfo().getMeasureDataTypes();
-  }
-
-  @Override
-  /**
-   * This method will add a record both key and value to list object
-   * it will keep track of how many record is processed, to handle limit scenario
-   * @param scanned result
-   *
-   */
-  public int collectData(AbstractScannedResult scannedResult, int batchSize) {
-    this.listBasedResult =
-        new ArrayList<>(batchSize);
-    boolean isMsrsPresent = measureDatatypes.length > 0;
-    ByteArrayWrapper wrapper = null;
-    // scan the record and add to list
-    ListBasedResultWrapper resultWrapper;
-    int rowCounter = 0;
-    while (scannedResult.hasNext() && rowCounter < batchSize) {
-      resultWrapper = new ListBasedResultWrapper();
-      if(tableBlockExecutionInfos.isDimensionsExistInQuery()) {
-        wrapper = new ByteArrayWrapper();
-        wrapper.setDictionaryKey(scannedResult.getDictionaryKeyArray());
-        wrapper.setNoDictionaryKeys(scannedResult.getNoDictionaryKeyArray());
-        wrapper.setComplexTypesKeys(scannedResult.getComplexTypeKeyArray());
-        resultWrapper.setKey(wrapper);
-      } else {
-        scannedResult.incrementCounter();
-      }
-      if(isMsrsPresent) {
-        Object[] msrValues = new Object[measureDatatypes.length];
-        fillMeasureData(msrValues, scannedResult);
-        resultWrapper.setValue(msrValues);
-      }
-      listBasedResult.add(resultWrapper);
-      rowCounter++;
-    }
-    return rowCounter;
-  }
-
-  private void fillMeasureData(Object[] msrValues, AbstractScannedResult scannedResult) {
-    for (short i = 0; i < measuresOrdinal.length; i++) {
-      // if measure exists is block then pass measure column
-      // data chunk to the collector
-      if (isMeasureExistsInCurrentBlock[i]) {
-        msrValues[i] =
-            getMeasureData(scannedResult.getMeasureChunk(measuresOrdinal[i]),
-                scannedResult.getCurrenrRowId(),measureDatatypes[i]);
-      } else {
-        // if not then get the default value and use that value in aggregation
-        msrValues[i] = measureDefaultValue[i];
-      }
-    }
-  }
-
-  private Object getMeasureData(MeasureColumnDataChunk dataChunk, int index, DataType dataType) {
-    if (!dataChunk.getNullValueIndexHolder().getBitSet().get(index)) {
-      Object msrVal;
-      switch (dataType) {
-        case LONG:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableLongValueByIndex(index);
-          break;
-        case DECIMAL:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableBigDecimalValueByIndex(index);
-          break;
-        default:
-          msrVal = dataChunk.getMeasureDataHolder().getReadableDoubleValueByIndex(index);
-      }
-      return DataTypeUtil.getMeasureDataBasedOnDataType(msrVal, dataType);
-    }
-    return null;
-  }
-
-  /**
-   * Below method will used to get the result
-   */
-  @Override public Result getCollectedResult() {
-    Result<List<ListBasedResultWrapper>, Object> result = new ListBasedResult();
-    if (!tableBlockExecutionInfos.isFixedKeyUpdateRequired()) {
-      updateKeyWithLatestBlockKeyGenerator();
-    }
-    result.addScannedResult(listBasedResult);
-    return result;
-  }
-
-
-
-  /**
-   * Below method will be used to update the fixed length key with the
-   * latest block key generator
-   *
-   * @return updated block
-   */
-  private void updateKeyWithLatestBlockKeyGenerator() {
-    try {
-      long[] data = null;
-      ByteArrayWrapper key = null;
-      for (int i = 0; i < listBasedResult.size(); i++) {
-        // get the key
-        key = listBasedResult.get(i).getKey();
-        // unpack the key with table block key generator
-        data = tableBlockExecutionInfos.getBlockKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(), tableBlockExecutionInfos.getMaskedByteForBlock());
-        // packed the key with latest block key generator
-        // and generate the masked key for that key
-        key.setDictionaryKey(QueryUtil
-            .getMaskedKey(restructureInfos.getKeyGenerator().generateKey(data),
-                restructureInfos.getMaxKey(), restructureInfos.getMaskByteRanges(),
-                restructureInfos.getMaskByteRanges().length));
-        listBasedResult.get(i).setKey(key);
-      }
-    } catch (KeyGenException e) {
-      LOGGER.error(e);
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutor.java
deleted file mode 100644
index 74713ba..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutor.java
+++ /dev/null
@@ -1,40 +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.carbondata.query.carbon.executor;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.model.QueryModel;
-
-/**
- * Interface for carbon query executor.
- * Will be used to execute the query based on the query model
- * and will return the iterator over query result
- */
-public interface QueryExecutor<E> {
-
-  /**
-   * Below method will be used to execute the query based on query model passed from driver
-   *
-   * @param queryModel query details
-   * @return query result iterator
-   * @throws QueryExecutionException if any failure while executing the query
-   */
-  CarbonIterator<E> execute(QueryModel queryModel) throws QueryExecutionException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java b/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.java
deleted file mode 100644
index fd1cf40..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/QueryExecutorFactory.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.carbondata.query.carbon.executor;
-
-import org.carbondata.query.carbon.executor.impl.DetailQueryExecutor;
-import org.carbondata.query.carbon.executor.impl.DetailRawRecordQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-
-/**
- * Factory class to get the query executor from RDD
- * This will return the executor based on query type
- */
-public class QueryExecutorFactory {
-
-  public static QueryExecutor getQueryExecutor(QueryModel queryModel) {
-    if (queryModel.isForcedDetailRawQuery()) {
-      return new DetailRawRecordQueryExecutor();
-    } else {
-      return new DetailQueryExecutor();
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/exception/QueryExecutionException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/exception/QueryExecutionException.java b/core/src/main/java/org/carbondata/query/carbon/executor/exception/QueryExecutionException.java
deleted file mode 100644
index 836009b..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/exception/QueryExecutionException.java
+++ /dev/null
@@ -1,96 +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.carbondata.query.carbon.executor.exception;
-
-import java.util.Locale;
-
-/**
- * Exception class for query execution
- *
- * @author Administrator
- */
-public class QueryExecutionException extends Exception {
-
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public QueryExecutionException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public QueryExecutionException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param t
-   */
-  public QueryExecutionException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java
deleted file mode 100644
index 37c4b68..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/AbstractQueryExecutor.java
+++ /dev/null
@@ -1,403 +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.carbondata.query.carbon.executor.impl;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.HashSet;
-import java.util.List;
-import java.util.Set;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.common.logging.impl.StandardLogService;
-import org.carbondata.core.carbon.datastore.BlockIndexStore;
-import org.carbondata.core.carbon.datastore.IndexKey;
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.datastore.exception.IndexBuilderException;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.core.keygenerator.KeyGenerator;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.QueryExecutor;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.AggregatorInfo;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-import org.carbondata.query.carbon.executor.infos.SortInfo;
-import org.carbondata.query.carbon.executor.util.QueryUtil;
-import org.carbondata.query.carbon.executor.util.RestructureUtil;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
-
-import org.apache.commons.lang3.ArrayUtils;
-
-/**
- * This class provides a skeletal implementation of the {@link QueryExecutor}
- * interface to minimize the effort required to implement this interface. This
- * will be used to prepare all the properties required for query execution
- */
-public abstract class AbstractQueryExecutor<E> implements QueryExecutor<E> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractQueryExecutor.class.getName());
-  /**
-   * holder for query properties which will be used to execute the query
-   */
-  protected QueryExecutorProperties queryProperties;
-
-  public AbstractQueryExecutor() {
-    queryProperties = new QueryExecutorProperties();
-  }
-
-  /**
-   * Below method will be used to fill the executor properties based on query
-   * model it will parse the query model and get the detail and fill it in
-   * query properties
-   *
-   * @param queryModel
-   */
-  protected void initQuery(QueryModel queryModel) throws QueryExecutionException {
-    StandardLogService.setThreadName(StandardLogService.getPartitionID(
-        queryModel.getAbsoluteTableIdentifier().getCarbonTableIdentifier().getTableName()),
-        queryModel.getQueryId());
-    LOGGER.info("Query will be executed on table: " + queryModel.getAbsoluteTableIdentifier()
-        .getCarbonTableIdentifier().getTableName());
-
-    QueryUtil.resolveQueryModel(queryModel);
-
-    // get the table blocks
-    try {
-      queryProperties.dataBlocks = BlockIndexStore.getInstance()
-          .loadAndGetBlocks(queryModel.getTableBlockInfos(),
-              queryModel.getAbsoluteTableIdentifier());
-    } catch (IndexBuilderException e) {
-      throw new QueryExecutionException(e);
-    }
-    //
-    // // updating the restructuring infos for the query
-    queryProperties.keyStructureInfo = getKeyStructureInfo(queryModel,
-        queryProperties.dataBlocks.get(queryProperties.dataBlocks.size() - 1).getSegmentProperties()
-            .getDimensionKeyGenerator());
-
-    // calculating the total number of aggeragted columns
-    int aggTypeCount = queryModel.getQueryMeasures().size();
-
-    int currentIndex = 0;
-    String[] aggTypes = new String[aggTypeCount];
-    DataType[] dataTypes = new DataType[aggTypeCount];
-
-    for (QueryMeasure carbonMeasure : queryModel.getQueryMeasures()) {
-      // adding the data type and aggregation type of all the measure this
-      // can be used
-      // to select the aggregator
-      aggTypes[currentIndex] = carbonMeasure.getAggregateFunction();
-      dataTypes[currentIndex] = carbonMeasure.getMeasure().getDataType();
-      currentIndex++;
-    }
-    queryProperties.measureDataTypes = dataTypes;
-    // as aggregation will be executed in following order
-    // 1.aggregate dimension expression
-    // 2. expression
-    // 3. query measure
-    // so calculating the index of the expression start index
-    // and measure column start index
-    queryProperties.aggExpressionStartIndex = queryModel.getQueryMeasures().size();
-    queryProperties.measureStartIndex = aggTypes.length - queryModel.getQueryMeasures().size();
-
-    // dictionary column unique column id to dictionary mapping
-    // which will be used to get column actual data
-    queryProperties.columnToDictionayMapping = QueryUtil
-        .getDimensionDictionaryDetail(queryModel.getQueryDimension(),
-            queryModel.getAbsoluteTableIdentifier());
-    queryModel.setColumnToDictionaryMapping(queryProperties.columnToDictionayMapping);
-    // setting the sort dimension index. as it will be updated while getting the sort info
-    // so currently setting it to default 0 means sort is not present in any dimension
-    queryProperties.sortDimIndexes = new byte[queryModel.getQueryDimension().size()];
-  }
-
-  /**
-   * Below method will be used to get the key structure info for the uqery
-   *
-   * @param queryModel   query model
-   * @param keyGenerator
-   * @return key structure info
-   */
-  private KeyStructureInfo getKeyStructureInfo(QueryModel queryModel, KeyGenerator keyGenerator) {
-    // getting the masked byte range for dictionary column
-    int[] maskByteRanges =
-        QueryUtil.getMaskedByteRange(queryModel.getQueryDimension(), keyGenerator);
-
-    // getting the masked bytes for query dimension dictionary column
-    int[] maskedBytes = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
-
-    // max key for the dictionary dimension present in the query
-    byte[] maxKey = null;
-    try {
-      // getting the max key which will be used to masked and get the
-      // masked key
-      maxKey = QueryUtil.getMaxKeyBasedOnDimensions(queryModel.getQueryDimension(), keyGenerator);
-    } catch (KeyGenException e) {
-      LOGGER.error(e, "problem while getting the max key");
-    }
-
-    KeyStructureInfo restructureInfos = new KeyStructureInfo();
-    restructureInfos.setKeyGenerator(keyGenerator);
-    restructureInfos.setMaskByteRanges(maskByteRanges);
-    restructureInfos.setMaskedBytes(maskedBytes);
-    restructureInfos.setMaxKey(maxKey);
-    return restructureInfos;
-  }
-
-  protected List<BlockExecutionInfo> getBlockExecutionInfos(QueryModel queryModel)
-      throws QueryExecutionException {
-    initQuery(queryModel);
-    List<BlockExecutionInfo> blockExecutionInfoList = new ArrayList<BlockExecutionInfo>();
-    // fill all the block execution infos for all the blocks selected in
-    // query
-    // and query will be executed based on that infos
-    for (int i = 0; i < queryProperties.dataBlocks.size(); i++) {
-      blockExecutionInfoList
-          .add(getBlockExecutionInfoForBlock(queryModel, queryProperties.dataBlocks.get(i)));
-    }
-    return blockExecutionInfoList;
-  }
-
-  /**
-   * Below method will be used to get the block execution info which is
-   * required to execute any block  based on query model
-   *
-   * @param queryModel query model from user query
-   * @param blockIndex block index
-   * @return block execution info
-   * @throws QueryExecutionException any failure during block info creation
-   */
-  protected BlockExecutionInfo getBlockExecutionInfoForBlock(QueryModel queryModel,
-      AbstractIndex blockIndex) throws QueryExecutionException {
-    BlockExecutionInfo blockExecutionInfo = new BlockExecutionInfo();
-    SegmentProperties segmentProperties = blockIndex.getSegmentProperties();
-    List<CarbonDimension> tableBlockDimensions = segmentProperties.getDimensions();
-    KeyGenerator blockKeyGenerator = segmentProperties.getDimensionKeyGenerator();
-
-    // below is to get only those dimension in query which is present in the
-    // table block
-    List<QueryDimension> updatedQueryDimension = RestructureUtil
-        .getUpdatedQueryDimension(queryModel.getQueryDimension(), tableBlockDimensions);
-    // TODO add complex dimension children
-    int[] maskByteRangesForBlock =
-        QueryUtil.getMaskedByteRange(updatedQueryDimension, blockKeyGenerator);
-    int[] maksedByte =
-        QueryUtil.getMaskedByte(blockKeyGenerator.getKeySizeInBytes(), maskByteRangesForBlock);
-    blockExecutionInfo.setDimensionsExistInQuery(updatedQueryDimension.size() > 0);
-    blockExecutionInfo.setDataBlock(blockIndex);
-    blockExecutionInfo.setBlockKeyGenerator(blockKeyGenerator);
-    // adding aggregation info for query
-    blockExecutionInfo.setAggregatorInfo(getAggregatorInfoForBlock(queryModel, blockIndex));
-
-    // setting the limit
-    blockExecutionInfo.setLimit(queryModel.getLimit());
-    // setting whether detail query or not
-    blockExecutionInfo.setDetailQuery(queryModel.isDetailQuery());
-    // setting whether raw record query or not
-    blockExecutionInfo.setRawRecordDetailQuery(queryModel.isForcedDetailRawQuery());
-    // setting the masked byte of the block which will be
-    // used to update the unpack the older block keys
-    blockExecutionInfo.setMaskedByteForBlock(maksedByte);
-    // total number dimension
-    blockExecutionInfo
-        .setTotalNumberDimensionBlock(segmentProperties.getDimensionOrdinalToBlockMapping().size());
-    blockExecutionInfo
-        .setTotalNumberOfMeasureBlock(segmentProperties.getMeasuresOrdinalToBlockMapping().size());
-    // to check whether older block key update is required or not
-    blockExecutionInfo.setFixedKeyUpdateRequired(
-        blockKeyGenerator.equals(queryProperties.keyStructureInfo.getKeyGenerator()));
-    IndexKey startIndexKey = null;
-    IndexKey endIndexKey = null;
-    if (null != queryModel.getFilterExpressionResolverTree()) {
-      // loading the filter executer tree for filter evaluation
-      blockExecutionInfo.setFilterExecuterTree(FilterUtil
-          .getFilterExecuterTree(queryModel.getFilterExpressionResolverTree(), segmentProperties));
-      List<IndexKey> listOfStartEndKeys = new ArrayList<IndexKey>(2);
-      FilterUtil.traverseResolverTreeAndGetStartAndEndKey(segmentProperties,
-          queryModel.getAbsoluteTableIdentifier(), queryModel.getFilterExpressionResolverTree(),
-          listOfStartEndKeys);
-      startIndexKey = listOfStartEndKeys.get(0);
-      endIndexKey = listOfStartEndKeys.get(1);
-    } else {
-      try {
-        startIndexKey = FilterUtil.prepareDefaultStartIndexKey(segmentProperties);
-        endIndexKey = FilterUtil.prepareDefaultEndIndexKey(segmentProperties);
-      } catch (KeyGenException e) {
-        throw new QueryExecutionException(e);
-      }
-    }
-    blockExecutionInfo.setFileType(
-        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
-    //setting the start index key of the block node
-    blockExecutionInfo.setStartKey(startIndexKey);
-    //setting the end index key of the block node
-    blockExecutionInfo.setEndKey(endIndexKey);
-    // expression dimensions
-    List<CarbonDimension> expressionDimensions =
-        new ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // expression measure
-    List<CarbonMeasure> expressionMeasures =
-        new ArrayList<CarbonMeasure>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    // setting all the dimension chunk indexes to be read from file
-    blockExecutionInfo.setAllSelectedDimensionBlocksIndexes(QueryUtil
-        .getDimensionsBlockIndexes(updatedQueryDimension,
-            segmentProperties.getDimensionOrdinalToBlockMapping(), expressionDimensions));
-    // setting all the measure chunk indexes to be read from file
-    blockExecutionInfo.setAllSelectedMeasureBlocksIndexes(QueryUtil
-        .getMeasureBlockIndexes(queryModel.getQueryMeasures(), expressionMeasures,
-            segmentProperties.getMeasuresOrdinalToBlockMapping()));
-    // setting the key structure info which will be required
-    // to update the older block key with new key generator
-    blockExecutionInfo.setKeyStructureInfo(queryProperties.keyStructureInfo);
-    // setting the size of fixed key column (dictionary column)
-    blockExecutionInfo.setFixedLengthKeySize(getKeySize(updatedQueryDimension, segmentProperties));
-    Set<Integer> dictionaryColumnBlockIndex = new HashSet<Integer>();
-    List<Integer> noDictionaryColumnBlockIndex = new ArrayList<Integer>();
-    // get the block index to be read from file for query dimension
-    // for both dictionary columns and no dictionary columns
-    QueryUtil.fillQueryDimensionsBlockIndexes(updatedQueryDimension,
-        segmentProperties.getDimensionOrdinalToBlockMapping(), dictionaryColumnBlockIndex,
-        noDictionaryColumnBlockIndex);
-    int[] queryDictionaruColumnBlockIndexes = ArrayUtils.toPrimitive(
-        dictionaryColumnBlockIndex.toArray(new Integer[dictionaryColumnBlockIndex.size()]));
-    // need to sort the dictionary column as for all dimension
-    // column key will be filled based on key order
-    Arrays.sort(queryDictionaruColumnBlockIndexes);
-    blockExecutionInfo.setDictionaryColumnBlockIndex(queryDictionaruColumnBlockIndexes);
-    // setting the no dictionary column block indexes
-    blockExecutionInfo.setNoDictionaryBlockIndexes(ArrayUtils.toPrimitive(
-        noDictionaryColumnBlockIndex.toArray(new Integer[noDictionaryColumnBlockIndex.size()])));
-    // setting column id to dictionary mapping
-    blockExecutionInfo.setColumnIdToDcitionaryMapping(queryProperties.columnToDictionayMapping);
-    // setting each column value size
-    blockExecutionInfo.setEachColumnValueSize(segmentProperties.getEachDimColumnValueSize());
-    try {
-      // to set column group and its key structure info which will be used
-      // to
-      // for getting the column group column data in case of final row
-      // and in case of dimension aggregation
-      blockExecutionInfo.setColumnGroupToKeyStructureInfo(
-          QueryUtil.getColumnGroupKeyStructureInfo(updatedQueryDimension, segmentProperties));
-    } catch (KeyGenException e) {
-      throw new QueryExecutionException(e);
-    }
-    return blockExecutionInfo;
-  }
-
-  /**
-   * This method will be used to get fixed key length size this will be used
-   * to create a row from column chunk
-   *
-   * @param queryDimension    query dimension
-   * @param blockMetadataInfo block metadata info
-   * @return key size
-   */
-  private int getKeySize(List<QueryDimension> queryDimension, SegmentProperties blockMetadataInfo) {
-    List<Integer> fixedLengthDimensionOrdinal =
-        new ArrayList<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    int counter = 0;
-    while (counter < queryDimension.size()) {
-      if (queryDimension.get(counter).getDimension().numberOfChild() > 0) {
-        counter += queryDimension.get(counter).getDimension().numberOfChild();
-        continue;
-      } else if (!CarbonUtil.hasEncoding(queryDimension.get(counter).getDimension().getEncoder(),
-          Encoding.DICTIONARY)) {
-        counter++;
-      } else {
-        fixedLengthDimensionOrdinal.add(queryDimension.get(counter).getDimension().getKeyOrdinal());
-        counter++;
-      }
-    }
-    int[] dictioanryColumnOrdinal = ArrayUtils.toPrimitive(
-        fixedLengthDimensionOrdinal.toArray(new Integer[fixedLengthDimensionOrdinal.size()]));
-    if (dictioanryColumnOrdinal.length > 0) {
-      return blockMetadataInfo.getFixedLengthKeySplitter()
-          .getKeySizeByBlock(dictioanryColumnOrdinal);
-    }
-    return 0;
-  }
-
-  /**
-   * Below method will be used to get the sort information which will be
-   * required during sorting the data on dimension column
-   *
-   * @param queryModel query model
-   * @return Sort infos
-   * @throws QueryExecutionException if problem while
-   */
-  protected SortInfo getSortInfos(QueryModel queryModel) throws QueryExecutionException {
-
-    // get the masked by range for order by dimension
-    int[][] maskedByteRangeForSorting = QueryUtil
-        .getMaskedByteRangeForSorting(queryModel.getSortDimension(),
-            queryProperties.keyStructureInfo.getKeyGenerator(),
-            queryProperties.keyStructureInfo.getMaskByteRanges());
-    // get masked key for sorting
-    byte[][] maksedKeyForSorting = QueryUtil.getMaksedKeyForSorting(queryModel.getSortDimension(),
-        queryProperties.keyStructureInfo.getKeyGenerator(), maskedByteRangeForSorting,
-        queryProperties.keyStructureInfo.getMaskByteRanges());
-    // fill sort dimension indexes
-    queryProperties.sortDimIndexes = QueryUtil
-        .getSortDimensionIndexes(queryModel.getSortDimension(), queryModel.getQueryDimension());
-    SortInfo sortInfos = new SortInfo();
-    sortInfos.setDimensionMaskKeyForSorting(maksedKeyForSorting);
-    sortInfos.setDimensionSortOrder(queryModel.getSortOrder());
-    sortInfos.setMaskedByteRangeForSorting(maskedByteRangeForSorting);
-    sortInfos.setSortDimensionIndex(queryProperties.sortDimIndexes);
-    sortInfos.setSortDimension(queryModel.getSortDimension());
-    return sortInfos;
-  }
-
-  /**
-   * Below method will be used to get the aggrgator info for the query
-   *
-   * @param queryModel query model
-   * @param tableBlock table block
-   * @return aggregator info
-   */
-  private AggregatorInfo getAggregatorInfoForBlock(QueryModel queryModel,
-      AbstractIndex tableBlock) {
-    // getting the aggregate infos which will be used during aggregation
-    AggregatorInfo aggregatorInfos = RestructureUtil
-        .getAggregatorInfos(queryModel.getQueryMeasures(),
-            tableBlock.getSegmentProperties().getMeasures());
-    // setting the index of expression in measure aggregators
-    aggregatorInfos.setExpressionAggregatorStartIndex(queryProperties.aggExpressionStartIndex);
-    // setting the index of measure columns in measure aggregators
-    aggregatorInfos.setMeasureAggregatorStartIndex(queryProperties.measureStartIndex);
-    // setting the measure aggregator for all aggregation function selected
-    // in query
-    aggregatorInfos.setMeasureDataTypes(queryProperties.measureDataTypes);
-    return aggregatorInfos;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
deleted file mode 100644
index 0255cbb..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailQueryExecutor.java
+++ /dev/null
@@ -1,46 +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.carbondata.query.carbon.executor.impl;
-
-import java.util.List;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
-import org.carbondata.query.carbon.result.iterator.DetailQueryResultIterator;
-import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPreparatorImpl;
-
-/**
- * Below class will be used to execute the detail query
- * For executing the detail query it will pass all the block execution
- * info to detail query result iterator and iterator will be returned
- */
-public class DetailQueryExecutor extends AbstractQueryExecutor {
-
-  @Override public CarbonIterator<Object[]> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    return new ChunkRowIterator(
-        new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
-            new DetailQueryResultPreparatorImpl(queryProperties, queryModel)));
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
deleted file mode 100644
index e72c638..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/DetailRawRecordQueryExecutor.java
+++ /dev/null
@@ -1,24 +0,0 @@
-package org.carbondata.query.carbon.executor.impl;
-
-import java.util.List;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.iterator.DetailQueryResultIterator;
-import org.carbondata.query.carbon.result.preparator.impl.RawQueryResultPreparatorImpl;
-
-/**
- * Executor for raw records, it does not parse to actual data
- */
-public class DetailRawRecordQueryExecutor extends AbstractQueryExecutor<BatchResult> {
-
-  @Override public CarbonIterator<BatchResult> execute(QueryModel queryModel)
-      throws QueryExecutionException {
-    List<BlockExecutionInfo> blockExecutionInfoList = getBlockExecutionInfos(queryModel);
-    return new DetailQueryResultIterator(blockExecutionInfoList, queryModel,
-        new RawQueryResultPreparatorImpl(queryProperties, queryModel));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java b/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
deleted file mode 100644
index ff42fbc..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/executor/impl/QueryExecutorProperties.java
+++ /dev/null
@@ -1,78 +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.carbondata.query.carbon.executor.impl;
-
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-
-/**
- * Holds all the properties required for query execution
- */
-public class QueryExecutorProperties {
-
-  /**
-   * list of blocks in which query will be executed
-   */
-  protected List<AbstractIndex> dataBlocks;
-
-  /**
-   * holds the information required for updating the order block
-   * dictionary key
-   */
-  public KeyStructureInfo keyStructureInfo;
-
-  /**
-   * as we have multiple type of column aggregation like
-   * dimension,expression,measure so this will be used to for getting the
-   * measure aggregation start index
-   */
-  public int measureStartIndex;
-
-  /**
-   * query like count(1),count(*) ,etc will used this parameter
-   */
-  public boolean isFunctionQuery;
-
-  /**
-   * aggExpressionStartIndex
-   */
-  public int aggExpressionStartIndex;
-
-  /**
-   * index of the dimension which is present in the order by
-   * in a query
-   */
-  public byte[] sortDimIndexes;
-
-  /**
-   * this will hold the information about the dictionary dimension
-   * which to
-   */
-  public Map<String, Dictionary> columnToDictionayMapping;
-
-  /**
-   * Measure datatypes
-   */
-  public DataType[] measureDataTypes;
-}


[54/56] [abbrv] incubator-carbondata git commit: Refactor carbon-core module for code clean up (#720)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/HdfsFileLock.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/HdfsFileLock.java b/processing/src/main/java/org/carbondata/lcm/locks/HdfsFileLock.java
new file mode 100644
index 0000000..f0c3ad5
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/HdfsFileLock.java
@@ -0,0 +1,89 @@
+/*
+ * 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.carbondata.lcm.locks;
+
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+
+/**
+ * This class is used to handle the HDFS File locking.
+ * This is acheived using the concept of acquiring the data out stream using Append option.
+ */
+public class HdfsFileLock extends AbstractCarbonLock {
+
+  /**
+   * location hdfs file location
+   */
+  private String location;
+
+  /**
+   * lockUsage is used to determine the type of the lock. according to this the lock
+   * folder will change.
+   */
+  private LockUsage lockUsage;
+
+  private DataOutputStream dataOutputStream;
+
+  /**
+   * @param location
+   * @param lockUsage
+   */
+  public HdfsFileLock(String location, LockUsage lockUsage) {
+    this.location = location;
+    this.lockUsage = lockUsage;
+    this.location = location + File.separator + this.lockUsage;
+    initRetry();
+  }
+
+  /* (non-Javadoc)
+   * @see org.carbondata.core.locks.ICarbonLock#lock()
+   */
+  @Override public boolean lock() {
+    try {
+      if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
+        FileFactory.createNewLockFile(location, FileFactory.getFileType(location));
+      }
+      dataOutputStream =
+          FileFactory.getDataOutputStreamUsingAppend(location, FileFactory.getFileType(location));
+
+      return true;
+
+    } catch (IOException e) {
+      return false;
+    }
+  }
+
+  /* (non-Javadoc)
+   * @see org.carbondata.core.locks.ICarbonLock#unlock()
+   */
+  @Override public boolean unlock() {
+    if (null != dataOutputStream) {
+      try {
+        dataOutputStream.close();
+      } catch (IOException e) {
+        return false;
+      }
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/ICarbonLock.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/ICarbonLock.java b/processing/src/main/java/org/carbondata/lcm/locks/ICarbonLock.java
new file mode 100644
index 0000000..bea57f7
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/ICarbonLock.java
@@ -0,0 +1,40 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.lcm.locks;
+
+/**
+ * Carbon Lock Interface which handles the locking and unlocking.
+ */
+public interface ICarbonLock {
+
+  /**
+   * Does the unlocking of the acquired lock.
+   *
+   * @return
+   */
+  boolean unlock();
+
+  /**
+   * This will acquire the lock and if it doesnt get then it will retry after the confiured time.
+   *
+   * @return
+   */
+  boolean lockWithRetries();
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java b/processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java
new file mode 100644
index 0000000..0a0fe62
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java
@@ -0,0 +1,162 @@
+/*
+ * 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.carbondata.lcm.locks;
+
+import java.io.File;
+import java.io.FileOutputStream;
+import java.io.IOException;
+import java.nio.channels.FileChannel;
+import java.nio.channels.FileLock;
+import java.nio.channels.OverlappingFileLockException;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+
+/**
+ * This class handles the file locking in the local file system.
+ * This will be handled using the file channel lock API.
+ */
+public class LocalFileLock extends AbstractCarbonLock {
+  /**
+   * location is the location of the lock file.
+   */
+  private String location;
+
+  /**
+   * lockUsage will determine the lock folder. so that similar locks will try to acquire
+   * same lock file.
+   */
+  private LockUsage lockUsage;
+
+  /**
+   * fileOutputStream of the local lock file
+   */
+  private FileOutputStream fileOutputStream;
+
+  /**
+   * channel is the FileChannel of the lock file.
+   */
+  private FileChannel channel;
+
+  /**
+   * fileLock NIO FileLock Object
+   */
+  private FileLock fileLock;
+
+  public static final String tmpPath;
+
+  private String cubeName;
+
+  private String schemaName;
+
+  /**
+   * LOGGER for  logging the messages.
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(LocalFileLock.class.getName());
+
+  static {
+    tmpPath = System.getProperty("java.io.tmpdir");
+  }
+
+  /**
+   * @param location
+   * @param lockUsage
+   */
+  public LocalFileLock(String location, LockUsage lockUsage) {
+    this.lockUsage = lockUsage;
+    location = location.replace("\\", "/");
+    String tempStr = location.substring(0, location.lastIndexOf('/'));
+    schemaName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
+
+    cubeName = location.substring(location.lastIndexOf('/') + 1, location.length());
+    this.location =
+        tmpPath + File.separator + schemaName + File.separator + cubeName + File.separator
+            + this.lockUsage;
+    initRetry();
+  }
+
+  /**
+   * Lock API for locking of the file channel of the lock file.
+   *
+   * @return
+   */
+  @Override public boolean lock() {
+    try {
+      String schemaFolderPath = tmpPath + File.separator + schemaName;
+      String cubeFolderPath = schemaFolderPath + File.separator + cubeName;
+      // create dir with schema name in tmp location.
+      if (!FileFactory.isFileExist(schemaFolderPath, FileFactory.getFileType(tmpPath))) {
+        FileFactory.mkdirs(schemaFolderPath, FileFactory.getFileType(tmpPath));
+      }
+
+      // create dir with cube name in tmp location.
+      if (!FileFactory.isFileExist(cubeFolderPath, FileFactory.getFileType(tmpPath))) {
+        FileFactory.mkdirs(cubeFolderPath, FileFactory.getFileType(tmpPath));
+      }
+      if (!FileFactory.isFileExist(location, FileFactory.getFileType(location))) {
+        FileFactory.createNewLockFile(location, FileFactory.getFileType(location));
+      }
+
+      fileOutputStream = new FileOutputStream(location);
+      channel = fileOutputStream.getChannel();
+      try {
+        fileLock = channel.tryLock();
+      } catch (OverlappingFileLockException e) {
+        return false;
+      }
+      if (null != fileLock) {
+        return true;
+      } else {
+        return false;
+      }
+    } catch (IOException e) {
+      return false;
+    }
+
+  }
+
+  /**
+   * Unlock API for unlocking of the acquired lock.
+   *
+   * @return
+   */
+  @Override public boolean unlock() {
+    boolean status;
+    try {
+      if (null != fileLock) {
+        fileLock.release();
+      }
+      status = true;
+    } catch (IOException e) {
+      status = false;
+    } finally {
+      if (null != fileOutputStream) {
+        try {
+          fileOutputStream.close();
+        } catch (IOException e) {
+          LOGGER.error(e.getMessage());
+        }
+      }
+    }
+    return status;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java b/processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java
new file mode 100644
index 0000000..e3eab01
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java
@@ -0,0 +1,29 @@
+/*
+ * 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.carbondata.lcm.locks;
+
+/**
+ * This enum is used to define the usecase of the lock.
+ * Each enum value is one specific lock case.
+ */
+public enum LockUsage {
+  METADATA_LOCK,
+  COMPACTION_LOCK;
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java b/processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java
new file mode 100644
index 0000000..ce5a62b
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/ZooKeeperLocking.java
@@ -0,0 +1,171 @@
+/*
+ * 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.carbondata.lcm.locks;
+
+import java.util.Collections;
+import java.util.List;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.constants.CarbonCommonConstants;
+
+import org.apache.zookeeper.CreateMode;
+import org.apache.zookeeper.KeeperException;
+import org.apache.zookeeper.ZooDefs.Ids;
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * For Handling the zookeeper locking implementation
+ */
+public class ZooKeeperLocking extends AbstractCarbonLock {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ZooKeeperLocking.class.getName());
+
+  /**
+   * zk is the zookeeper client instance
+   */
+  private static ZooKeeper zk;
+
+  /**
+   * zooKeeperLocation is the location in the zoo keeper file system where the locks will be
+   * maintained.
+   */
+  private static final String zooKeeperLocation = CarbonCommonConstants.ZOOKEEPER_LOCATION;
+
+  /**
+   * Unique folder for each cube with SchemaName_CubeName
+   */
+  private final String tableIdFolder;
+
+  /**
+   * lockName is the name of the lock to use. This name should be same for every process that want
+   * to share the same lock
+   */
+  private String lockName;
+
+  /**
+   * lockPath is the unique path created for the each instance of the carbon lock.
+   */
+  private String lockPath;
+
+  private String lockTypeFolder;
+
+  /**
+   * @param lockUsage
+   */
+  public ZooKeeperLocking(String location, LockUsage lockUsage) {
+    this.lockName = CarbonCommonConstants.ZOOKEEPER_LOCK;
+    this.lockTypeFolder = zooKeeperLocation;
+    location = location.replace("\\", "/");
+    String tempStr = location.substring(0, location.lastIndexOf('/'));
+    String schemaName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
+
+    String cubeName = location.substring(location.lastIndexOf('/') + 1, location.length());
+
+    this.tableIdFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + schemaName
+        + '.' + cubeName;
+
+    zk = ZookeeperInit.getInstance().getZookeeper();
+
+    this.lockTypeFolder = zooKeeperLocation + CarbonCommonConstants.FILE_SEPARATOR + schemaName
+        + '.' + cubeName + CarbonCommonConstants.FILE_SEPARATOR
+        + lockUsage.toString();
+    try {
+      createBaseNode();
+      // if exists returns null then path doesnt exist. so creating.
+      if (null == zk.exists(this.tableIdFolder, true)) {
+        zk.create(this.tableIdFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+      }
+      // if exists returns null then path doesnt exist. so creating.
+      if (null == zk.exists(this.lockTypeFolder, true)) {
+        zk.create(this.lockTypeFolder, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+      }
+    } catch (KeeperException | InterruptedException e) {
+      LOGGER.error(e, e.getMessage());
+    }
+    initRetry();
+  }
+
+  /**
+   * Creating a znode in which all the znodes (lock files )are maintained.
+   */
+  private void createBaseNode() throws KeeperException, InterruptedException {
+    if (null == zk.exists(zooKeeperLocation, true)) {
+      // creating a znode in which all the znodes (lock files )are maintained.
+      zk.create(zooKeeperLocation, new byte[1], Ids.OPEN_ACL_UNSAFE, CreateMode.PERSISTENT);
+    }
+  }
+
+  /**
+   * Handling of the locking mechanism using zoo keeper.
+   */
+  @Override public boolean lock() {
+    try {
+      // create the lock file with lockName.
+      lockPath =
+          zk.create(this.lockTypeFolder + CarbonCommonConstants.FILE_SEPARATOR + lockName, null,
+              Ids.OPEN_ACL_UNSAFE, CreateMode.EPHEMERAL_SEQUENTIAL);
+
+      // get the children present in zooKeeperLocation.
+      List<String> nodes = zk.getChildren(this.lockTypeFolder, null);
+
+      // sort the childrens
+      Collections.sort(nodes);
+
+      // here the logic is , for each lock request zookeeper will create a file ending with
+      // incremental digits.
+      // so first request will be 00001 next is 00002 and so on.
+      // if the current request is 00002 and already one previous request(00001) is present then get
+      // children will give both nodes.
+      // after the sort we are checking if the lock path is first or not .if it is first then lock
+      // has been acquired.
+
+      if (lockPath.endsWith(nodes.get(0))) {
+        return true;
+      } else {
+        // if locking failed then deleting the created lock as next time again new lock file will be
+        // created.
+        zk.delete(lockPath, -1);
+        return false;
+      }
+    } catch (KeeperException | InterruptedException e) {
+      LOGGER.error(e, e.getMessage());
+      return false;
+    }
+  }
+
+  /**
+   * @return status where lock file is unlocked or not.
+   */
+  @Override public boolean unlock() {
+    try {
+      // exists will return null if the path doesn't exists.
+      if (null != zk.exists(lockPath, true)) {
+        zk.delete(lockPath, -1);
+        lockPath = null;
+      }
+    } catch (KeeperException | InterruptedException e) {
+      LOGGER.error(e, e.getMessage());
+      return false;
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/ZookeeperInit.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/ZookeeperInit.java b/processing/src/main/java/org/carbondata/lcm/locks/ZookeeperInit.java
new file mode 100644
index 0000000..e7ac49a
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/ZookeeperInit.java
@@ -0,0 +1,64 @@
+package org.carbondata.lcm.locks;
+
+import java.io.IOException;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+
+import org.apache.zookeeper.WatchedEvent;
+import org.apache.zookeeper.Watcher;
+import org.apache.zookeeper.ZooKeeper;
+
+/**
+ * This is a singleton class for initialization of zookeeper client.
+ */
+public class ZookeeperInit {
+
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ZookeeperInit.class.getName());
+
+  private static ZookeeperInit zooKeeperInit;
+  /**
+   * zk is the zookeeper client instance
+   */
+  private ZooKeeper zk;
+
+  private ZookeeperInit(String zooKeeperUrl) {
+
+    int sessionTimeOut = 100000;
+    try {
+      zk = new ZooKeeper(zooKeeperUrl, sessionTimeOut, new DummyWatcher());
+
+    } catch (IOException e) {
+      LOGGER.error(e.getMessage());
+    }
+
+  }
+
+  public static ZookeeperInit getInstance(String zooKeeperUrl) {
+
+    if (null == zooKeeperInit) {
+      synchronized (ZookeeperInit.class) {
+        if (null == zooKeeperInit) {
+          LOGGER.info("Initiating Zookeeper client.");
+          zooKeeperInit = new ZookeeperInit(zooKeeperUrl);
+        }
+      }
+    }
+    return zooKeeperInit;
+
+  }
+
+  public static ZookeeperInit getInstance() {
+    return zooKeeperInit;
+  }
+
+  public ZooKeeper getZookeeper() {
+    return zk;
+  }
+
+  private static class DummyWatcher implements Watcher {
+    public void process(WatchedEvent event) {
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java b/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
index f8ac76d..20f6e7e 100644
--- a/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
@@ -32,15 +32,15 @@ import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.fileperations.AtomicFileOperations;
-import org.carbondata.core.datastorage.store.fileperations.AtomicFileOperationsImpl;
-import org.carbondata.core.datastorage.store.fileperations.FileWriteOperation;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.load.LoadMetadataDetails;
-import org.carbondata.core.locks.CarbonLockFactory;
-import org.carbondata.core.locks.ICarbonLock;
-import org.carbondata.core.locks.LockUsage;
 import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.lcm.fileoperations.AtomicFileOperations;
+import org.carbondata.lcm.fileoperations.AtomicFileOperationsImpl;
+import org.carbondata.lcm.fileoperations.FileWriteOperation;
+import org.carbondata.lcm.locks.CarbonLockFactory;
+import org.carbondata.lcm.locks.ICarbonLock;
+import org.carbondata.lcm.locks.LockUsage;
 
 import com.google.gson.Gson;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
index d680265..dcd5e19 100644
--- a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@ -37,17 +37,17 @@ import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.file.manager.composite.FileData;
-import org.carbondata.core.file.manager.composite.FileManager;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
 import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 import org.carbondata.processing.datatypes.GenericDataType;
+import org.carbondata.processing.mdkeygen.file.FileData;
+import org.carbondata.processing.mdkeygen.file.FileManager;
+import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.carbondata.processing.store.CarbonDataFileAttributes;
 import org.carbondata.processing.store.CarbonFactDataHandlerColumnar;
 import org.carbondata.processing.store.CarbonFactDataHandlerModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileData.java b/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileData.java
new file mode 100644
index 0000000..aee802c
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileData.java
@@ -0,0 +1,74 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.carbondata.processing.mdkeygen.file;
+
+import org.carbondata.core.writer.HierarchyValueWriterForCSV;
+
+public class FileData extends FileManager {
+
+  /**
+   * Store Path
+   */
+  private String storePath;
+
+  /**
+   * hierarchyValueWriter
+   */
+  private HierarchyValueWriterForCSV hierarchyValueWriter;
+
+  public FileData(String fileName, String storePath) {
+    this.fileName = fileName;
+    this.storePath = storePath;
+  }
+
+  /**
+   * @return Returns the fileName.
+   */
+  public String getFileName() {
+    return fileName;
+  }
+
+  /**
+   * @return Returns the storePath.
+   */
+  public String getStorePath() {
+    return storePath;
+  }
+
+  /**
+   * get Hierarchy Value writer
+   *
+   * @return
+   */
+  public HierarchyValueWriterForCSV getHierarchyValueWriter() {
+    return hierarchyValueWriter;
+  }
+
+  /**
+   * Set Hierarchy Value Writer.
+   *
+   * @param hierarchyValueWriter
+   */
+  public void setHierarchyValueWriter(HierarchyValueWriterForCSV hierarchyValueWriter) {
+    this.hierarchyValueWriter = hierarchyValueWriter;
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileManager.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileManager.java b/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileManager.java
new file mode 100644
index 0000000..7aece32
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/file/FileManager.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.carbondata.processing.mdkeygen.file;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+
+public class FileManager implements IFileManagerComposite {
+  /**
+   * listOfFileData, composite parent which holds the different objects
+   */
+  protected List<IFileManagerComposite> listOfFileData =
+      new ArrayList<IFileManagerComposite>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+
+  protected String fileName;
+
+  @Override public void add(IFileManagerComposite customData) {
+    listOfFileData.add(customData);
+  }
+
+  @Override public void remove(IFileManagerComposite customData) {
+    listOfFileData.remove(customData);
+
+  }
+
+  @Override public IFileManagerComposite get(int i) {
+    return listOfFileData.get(i);
+  }
+
+  /**
+   * Renames the File/Folders
+   */
+  public boolean rename(IFileManagerComposite composite) {
+    return false;
+  }
+
+  @Override public void setName(String name) {
+    this.fileName = name;
+  }
+
+  /**
+   * Return the size
+   */
+  public int size() {
+    return listOfFileData.size();
+  }
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/mdkeygen/file/IFileManagerComposite.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/mdkeygen/file/IFileManagerComposite.java b/processing/src/main/java/org/carbondata/processing/mdkeygen/file/IFileManagerComposite.java
new file mode 100644
index 0000000..47008d2
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/file/IFileManagerComposite.java
@@ -0,0 +1,59 @@
+/*
+ * 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.carbondata.processing.mdkeygen.file;
+
+public interface IFileManagerComposite {
+  /**
+   * Add the data which can be either row Folder(Composite) or File
+   *
+   * @param customData
+   */
+  void add(IFileManagerComposite customData);
+
+  /**
+   * Remove the CustomData type object from the IFileManagerComposite object hierarchy.
+   *
+   * @param customData
+   */
+  void remove(IFileManagerComposite customData);
+
+  /**
+   * get the CustomData type object name
+   *
+   * @return CustomDataIntf type
+   */
+  IFileManagerComposite get(int i);
+
+  /**
+   * set the CustomData type object name
+   *
+   * @param name
+   */
+  void setName(String name);
+
+  /**
+   * Get the size
+   *
+   * @return
+   */
+  int size();
+
+}
+

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/merger/util/CarbonSliceMergerUtil.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/merger/util/CarbonSliceMergerUtil.java b/processing/src/main/java/org/carbondata/processing/merger/util/CarbonSliceMergerUtil.java
deleted file mode 100644
index bb0af50..0000000
--- a/processing/src/main/java/org/carbondata/processing/merger/util/CarbonSliceMergerUtil.java
+++ /dev/null
@@ -1,366 +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.carbondata.processing.merger.util;
-
-import java.io.BufferedInputStream;
-import java.io.BufferedOutputStream;
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.InputStream;
-import java.io.OutputStream;
-import java.nio.ByteBuffer;
-import java.nio.channels.FileChannel;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.HashMap;
-import java.util.LinkedHashMap;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.core.writer.ByteArrayHolder;
-import org.carbondata.processing.merger.exeception.SliceMergerException;
-
-public final class CarbonSliceMergerUtil {
-  /**
-   * Comment for <code>LOGGER</code>
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonSliceMergerUtil.class.getName());
-
-  private CarbonSliceMergerUtil() {
-
-  }
-
-  /**
-   * Below method will be used to get the file map
-   * Map will contain Key as a type of file(File Name) and its list of files
-   *
-   * @param sliceFiles slice files
-   * @return file map
-   */
-  public static Map<String, List<CarbonFile>> getFileMap(CarbonFile[][] sliceFiles) {
-    Map<String, List<CarbonFile>> filesMap = new LinkedHashMap<String, List<CarbonFile>>();
-    for (int i = 0; i < sliceFiles.length; i++) {
-      for (int j = 0; j < sliceFiles[i].length; j++) {
-        String fileName = sliceFiles[i][j].getName();
-        List<CarbonFile> fileList = filesMap.get(fileName);
-        if (null == fileList) {
-          fileList = new ArrayList<CarbonFile>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-          fileList.add(sliceFiles[i][j]);
-        } else {
-          fileList.add(sliceFiles[i][j]);
-        }
-        filesMap.put(fileName, fileList);
-      }
-    }
-    return filesMap;
-  }
-
-  /**
-   * This method will be used for copy file from source to destination
-   * location
-   *
-   * @param sourceLocation      source path
-   * @param desTinationLocation destination path
-   * @throws IOException if any problem while  reading or writing the files
-   */
-  public static void copyFile(CarbonFile sourceLocation, String desTinationLocation)
-      throws IOException {
-
-    InputStream inputStream = null;
-    OutputStream outputStream = null;
-    try {
-      inputStream = FileFactory.getDataInputStream(sourceLocation.getAbsolutePath(),
-          FileFactory.getFileType(sourceLocation.getAbsolutePath()));
-      outputStream = FileFactory
-          .getDataOutputStream(desTinationLocation, FileFactory.getFileType(desTinationLocation),
-              10240, true);
-
-      copyFile(inputStream, outputStream);
-    } finally {
-      CarbonUtil.closeStreams(inputStream, outputStream);
-    }
-  }
-
-  /**
-   * This metod copy the multiple level files and merge into single file.
-   *
-   * @param filesToMerge
-   * @param destFile
-   * @throws IOException
-   */
-  public static void copyMultipleFileToSingleFile(List<File> filesToMerge, File destFile)
-      throws IOException {
-
-    InputStream inputStream = null;
-    OutputStream outputStream = null;
-    try {
-      outputStream = new BufferedOutputStream(new FileOutputStream(destFile, true));
-
-      for (File toMerge : filesToMerge) {
-        inputStream = new BufferedInputStream(new FileInputStream(toMerge));
-        copyFileWithoutClosingOutputStream(inputStream, outputStream);
-      }
-    } finally {
-      CarbonUtil.closeStreams(inputStream, outputStream);
-    }
-
-  }
-
-  /**
-   * This method reads the hierarchy file, sort the Mdkey and write into the destination
-   * file.
-   *
-   * @param filesToMerge
-   * @param destFile
-   * @throws IOException
-   */
-  public static void mergeHierarchyFiles(List<File> filesToMerge, File destFile, int keySizeInBytes)
-      throws IOException {
-    List<ByteArrayHolder> holder =
-        new ArrayList<ByteArrayHolder>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-
-    for (File hierFiles : filesToMerge) {
-      readHierarchyFile(hierFiles, keySizeInBytes, holder);
-    }
-
-    Collections.sort(holder);
-
-    FileOutputStream fos = null;
-    FileChannel outPutFileChannel = null;
-
-    try {
-
-      boolean isFileCreated = false;
-      if (!destFile.exists()) {
-        isFileCreated = destFile.createNewFile();
-
-        if (!isFileCreated) {
-          throw new IOException("unable to create file" + destFile.getAbsolutePath());
-        }
-      }
-
-      fos = new FileOutputStream(destFile);
-
-      outPutFileChannel = fos.getChannel();
-      for (ByteArrayHolder arrayHolder : holder) {
-        try {
-          writeIntoHierarchyFile(arrayHolder.getMdKey(), arrayHolder.getPrimaryKey(),
-              outPutFileChannel);
-        } catch (SliceMergerException e) {
-          LOGGER.error("Unable to write hierarchy file");
-          throw new IOException(e);
-        }
-
-      }
-
-    } finally {
-      CarbonUtil.closeStreams(outPutFileChannel, fos);
-    }
-
-  }
-
-  private static void writeIntoHierarchyFile(byte[] bytes, int primaryKey,
-      FileChannel outPutFileChannel) throws SliceMergerException {
-
-    ByteBuffer byteBuffer = storeValueInCache(bytes, primaryKey);
-
-    try {
-      byteBuffer.flip();
-      outPutFileChannel.write(byteBuffer);
-    } catch (IOException e) {
-      throw new SliceMergerException(
-          "Error while writting in the hierarchy mapping file at the merge step", e);
-    }
-  }
-
-  private static ByteBuffer storeValueInCache(byte[] bytes, int primaryKey) {
-
-    // adding 4 to store the total length of the row at the beginning
-    ByteBuffer buffer = ByteBuffer.allocate(bytes.length + 4);
-
-    buffer.put(bytes);
-    buffer.putInt(primaryKey);
-
-    return buffer;
-  }
-
-  /**
-   * setHeirAndKeySizeMap
-   *
-   * @param heirAndKeySize void
-   */
-  public static Map<String, Integer> getHeirAndKeySizeMap(String heirAndKeySize) {
-    String[] split = heirAndKeySize.split(CarbonCommonConstants.AMPERSAND_SPC_CHARACTER);
-    String[] split2 = null;
-    Map<String, Integer> heirAndKeySizeMap = new HashMap<String, Integer>(split.length);
-    for (int i = 0; i < split.length; i++) {
-      split2 = split[i].split(CarbonCommonConstants.COLON_SPC_CHARACTER);
-      heirAndKeySizeMap.put(split2[0], Integer.parseInt(split2[1]));
-    }
-
-    return heirAndKeySizeMap;
-  }
-
-  private static void readHierarchyFile(File hierarchyFile, int keySizeInBytes,
-      List<ByteArrayHolder> byteArrayHolder) throws IOException {
-    int rowLength = keySizeInBytes + 4;
-    FileInputStream inputStream = null;
-    FileChannel fileChannel = null;
-
-    inputStream = new FileInputStream(hierarchyFile);
-    fileChannel = inputStream.getChannel();
-
-    long size = fileChannel.size();
-    ByteBuffer rowlengthToRead = ByteBuffer.allocate(rowLength);
-    try {
-      while (fileChannel.position() < size) {
-        fileChannel.read(rowlengthToRead);
-        rowlengthToRead.rewind();
-
-        byte[] mdKey = new byte[keySizeInBytes];
-        rowlengthToRead.get(mdKey);
-        int primaryKey = rowlengthToRead.getInt();
-        byteArrayHolder.add(new ByteArrayHolder(mdKey, primaryKey));
-        rowlengthToRead.clear();
-      }
-    } finally {
-      CarbonUtil.closeStreams(fileChannel, inputStream);
-    }
-
-  }
-
-  /**
-   * This method will copy input stream to output stream it will copy file to
-   * destination
-   *
-   * @param stream    InputStream
-   * @param outStream outStream
-   * @throws IOException
-   * @throws IOException IOException
-   */
-  private static void copyFile(InputStream stream, OutputStream outStream) throws IOException {
-    int len;
-    final byte[] buffer = new byte[CarbonCommonConstants.BYTEBUFFER_SIZE];
-    try {
-      for (; ; ) {
-        len = stream.read(buffer);
-        if (len == -1) {
-          return;
-        }
-        outStream.write(buffer, 0, len);
-      }
-    } catch (IOException e) {
-      throw e;
-    } finally {
-      CarbonUtil.closeStreams(stream, outStream);
-    }
-  }
-
-  /**
-   * This method will copy input stream to output stream it will copy file to
-   * destination and will not close the outputStream.
-   *
-   * @param stream    InputStream
-   * @param outStream outStream
-   * @throws IOException
-   * @throws IOException IOException
-   */
-  private static void copyFileWithoutClosingOutputStream(InputStream stream, OutputStream outStream)
-      throws IOException {
-
-    final byte[] buffer = new byte[CarbonCommonConstants.BYTEBUFFER_SIZE];
-    int len;
-    try {
-      for (; ; ) {
-        len = stream.read(buffer);
-        if (len == -1) {
-          return;
-        }
-        outStream.write(buffer, 0, len);
-      }
-    } catch (IOException e) {
-      throw e;
-    } finally {
-      CarbonUtil.closeStreams(stream);
-    }
-  }
-
-  /**
-   * compare
-   *
-   * @param byte1
-   * @param byte2
-   * @return int
-   */
-  public static int compare(byte[] byte1, byte[] byte2) {
-    int cmp = 0;
-    int length = byte1.length;
-    for (int i = 0; i < length; i++) {
-      int a = (byte1[i] & 0xff);
-      int b = (byte2[i] & 0xff);
-      cmp = a - b;
-      if (cmp != 0) {
-        cmp = cmp < 0 ? -1 : 1;
-        break;
-      }
-    }
-    return cmp;
-  }
-
-  public static File decryptEncyptedFile(File memberFile) throws SliceMergerException {
-    String filePath = memberFile.getAbsolutePath() + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-    return new File(filePath);
-  }
-
-  /**
-   * below method will be used to get the files
-   *
-   * @param sliceLocation slocation locations
-   * @return sorted files
-   */
-  public static CarbonFile[] getSortedPathForFiles(String sliceLocation) {
-    FileType fileType = FileFactory.getFileType(sliceLocation);
-    CarbonFile storeFolder = FileFactory.getCarbonFile(sliceLocation, fileType);
-
-    CarbonFile[] files = storeFolder.listFiles(new CarbonFileFilter() {
-
-      @Override public boolean accept(CarbonFile pathname) {
-        if (!(pathname.isDirectory()) && pathname.getName().endsWith(".hierarchy")) {
-          return true;
-        }
-        return false;
-      }
-    });
-
-    return CarbonUtil.getSortedFileList(files);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
index ed7bb23..0c91ffe 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -48,12 +48,11 @@ import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.columnar.BlockIndexerStorageForInt;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import org.carbondata.core.datastorage.store.columnar.IndexStorage;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.carbondata.core.datastorage.store.dataholder.CarbonWriteDataHolder;
 import org.carbondata.core.datastorage.util.StoreFactory;
-import org.carbondata.core.file.manager.composite.FileManager;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
@@ -64,8 +63,9 @@ import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.DataTypeUtil;
 import org.carbondata.core.util.ValueCompressionUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 import org.carbondata.processing.datatypes.GenericDataType;
+import org.carbondata.processing.mdkeygen.file.FileManager;
+import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.carbondata.processing.store.colgroup.ColGroupBlockStorage;
 import org.carbondata.processing.store.colgroup.ColGroupDataHolder;
 import org.carbondata.processing.store.colgroup.ColGroupMinMax;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
index a3f7b4f..0325068 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -21,7 +21,7 @@ package org.carbondata.processing.store;
 
 import java.util.Map;
 
-import org.carbondata.core.vo.ColumnGroupModel;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import org.carbondata.processing.datatypes.GenericDataType;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
index e62c8f2..31dec3e 100644
--- a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
+++ b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
@@ -18,7 +18,7 @@
  */
 package org.carbondata.processing.store.colgroup;
 
-import org.carbondata.core.vo.ColumnGroupModel;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 
 /**
  * This will hold column group data.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
index a41bf08..0ae0d7f 100644
--- a/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
+++ b/processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
@@ -25,11 +25,11 @@ import java.util.Set;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.carbondata.core.util.ByteUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 
 /**
  * it gives min max of each column of column group

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java b/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
index a0fa842..48b9336 100644
--- a/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
+++ b/processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
@@ -49,8 +49,6 @@ import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.file.manager.composite.FileData;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
 import org.carbondata.core.metadata.BlockletInfoColumnar;
 import org.carbondata.core.util.CarbonMergerUtil;
 import org.carbondata.core.util.CarbonMetadataUtil;
@@ -58,6 +56,8 @@ import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.writer.CarbonFooterWriter;
 import org.carbondata.format.FileFooter;
+import org.carbondata.processing.mdkeygen.file.FileData;
+import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.carbondata.processing.store.CarbonDataFileAttributes;
 import org.carbondata.processing.store.writer.exception.CarbonDataWriterException;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java b/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
index 6177385..ce941dd 100644
--- a/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
+++ b/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
@@ -29,10 +29,10 @@ import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.columnar.IndexStorage;
 import org.carbondata.core.datastorage.store.compression.SnappyCompression.SnappyByteCompression;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
 import org.carbondata.core.keygenerator.mdkey.NumberCompressor;
 import org.carbondata.core.metadata.BlockletInfoColumnar;
 import org.carbondata.core.util.CarbonProperties;
+import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.carbondata.processing.store.CarbonDataFileAttributes;
 import org.carbondata.processing.store.colgroup.ColGroupBlockStorage;
 import org.carbondata.processing.store.writer.exception.CarbonDataWriterException;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
index 70b6199..6ef58fc 100644
--- a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
+++ b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedDimSurrogateKeyGen.java
@@ -34,10 +34,10 @@ import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.cache.dictionary.Dictionary;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.writer.HierarchyValueWriterForCSV;
 import org.carbondata.processing.datatypes.GenericDataType;
+import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.carbondata.processing.schema.metadata.ArrayWrapper;
 import org.carbondata.processing.schema.metadata.ColumnsInfo;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
index e40bd25..57b330b 100644
--- a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
@@ -50,9 +50,6 @@ import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.file.manager.composite.FileData;
-import org.carbondata.core.file.manager.composite.FileManager;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
@@ -64,6 +61,9 @@ import org.carbondata.core.writer.ByteArrayHolder;
 import org.carbondata.core.writer.HierarchyValueWriterForCSV;
 import org.carbondata.processing.dataprocessor.manager.CarbonDataProcessorManager;
 import org.carbondata.processing.datatypes.GenericDataType;
+import org.carbondata.processing.mdkeygen.file.FileData;
+import org.carbondata.processing.mdkeygen.file.FileManager;
+import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.carbondata.processing.schema.metadata.ColumnSchemaDetails;
 import org.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper;
 import org.carbondata.processing.schema.metadata.ColumnsInfo;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
index e2ca310..a78e40a 100644
--- a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
+++ b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
@@ -39,9 +39,6 @@ import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.file.manager.composite.FileData;
-import org.carbondata.core.file.manager.composite.FileManager;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.util.CarbonProperties;
@@ -49,6 +46,9 @@ import org.carbondata.core.util.CarbonUtilException;
 import org.carbondata.core.writer.ByteArrayHolder;
 import org.carbondata.core.writer.HierarchyValueWriterForCSV;
 import org.carbondata.processing.datatypes.GenericDataType;
+import org.carbondata.processing.mdkeygen.file.FileData;
+import org.carbondata.processing.mdkeygen.file.FileManager;
+import org.carbondata.processing.mdkeygen.file.IFileManagerComposite;
 import org.carbondata.processing.schema.metadata.ColumnSchemaDetails;
 import org.carbondata.processing.schema.metadata.ColumnSchemaDetailsWrapper;
 import org.carbondata.processing.schema.metadata.ColumnsInfo;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java b/processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java
new file mode 100644
index 0000000..e41ed99
--- /dev/null
+++ b/processing/src/test/java/org/carbondata/lcm/locks/LocalFileLockTest.java
@@ -0,0 +1,48 @@
+/**
+ *
+ */
+package org.carbondata.lcm.locks;
+
+import java.io.File;
+
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+import org.pentaho.di.core.util.Assert;
+
+/**
+ * Test class to test the functionality of the local file locking.
+ *
+ * @author Administrator
+ */
+public class LocalFileLockTest {
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Before public void setUp() throws Exception {
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @After public void tearDown() throws Exception {
+  }
+
+  @Test public void testingLocalFileLockingByAcquiring2Locks() {
+
+    LocalFileLock localLock1 =
+        new LocalFileLock((new File(".").getAbsolutePath()) + "/src/test/resources",
+            LockUsage.METADATA_LOCK);
+    Assert.assertTrue(localLock1.lock());
+    LocalFileLock localLock2 =
+        new LocalFileLock((new File(".").getAbsolutePath()) + "/src/test/resources",
+            LockUsage.METADATA_LOCK);
+    Assert.assertTrue(!localLock2.lock());
+
+    Assert.assertTrue(localLock1.unlock());
+    Assert.assertTrue(localLock2.lock());
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java b/processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java
new file mode 100644
index 0000000..87616e7
--- /dev/null
+++ b/processing/src/test/java/org/carbondata/lcm/locks/ZooKeeperLockingTest.java
@@ -0,0 +1,125 @@
+/**
+ *
+ */
+package org.carbondata.lcm.locks;
+
+import java.io.File;
+import java.io.IOException;
+import java.net.ServerSocket;
+import java.util.Properties;
+
+import org.carbondata.core.util.CarbonProperties;
+
+import mockit.NonStrictExpectations;
+import org.apache.zookeeper.server.ServerConfig;
+import org.apache.zookeeper.server.ZooKeeperServerMain;
+import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * @author Administrator
+ */
+public class ZooKeeperLockingTest {
+
+  int freePort;
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @Before public void setUp() throws Exception {
+    Properties startupProperties = new Properties();
+    startupProperties.setProperty("dataDir", (new File("./target").getAbsolutePath()));
+    startupProperties.setProperty("dataLogDir", (new File("./target").getAbsolutePath()));
+    freePort = findFreePort();
+    startupProperties.setProperty("clientPort", "" + freePort);
+    QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig();
+    try {
+      quorumConfiguration.parseProperties(startupProperties);
+    } catch (Exception e) {
+      throw new RuntimeException(e);
+    }
+
+    final ZooKeeperServerMain zooKeeperServer = new ZooKeeperServerMain();
+    final ServerConfig configuration = new ServerConfig();
+    configuration.readFrom(quorumConfiguration);
+    new Thread() {
+      public void run() {
+        try {
+          zooKeeperServer.runFromConfig(configuration);
+        } catch (IOException e) {
+          System.out.println("ZooKeeper failure");
+        }
+      }
+    }.start();
+  }
+
+  /**
+   * @throws java.lang.Exception
+   */
+  @After public void tearDown() throws Exception {
+  }
+
+  @Test public void testZooKeeperLockingByTryingToAcquire2Locks()
+      throws IllegalArgumentException, IllegalAccessException, NoSuchFieldException,
+      SecurityException {
+
+    final CarbonProperties cp = CarbonProperties.getInstance();
+    new NonStrictExpectations(cp) {
+      {
+        cp.getProperty("/CarbonLocks");
+        result = "/carbontests";
+        cp.getProperty("spark.deploy.zookeeper.url");
+        result = "127.0.0.1:" + freePort;
+      }
+    };
+
+    ZookeeperInit zki = ZookeeperInit.getInstance("127.0.0.1:" + freePort);
+
+    ZooKeeperLocking zkl =
+        new ZooKeeperLocking("D:/carbondata/examples/target/store/default/t3/Metadata",
+            LockUsage.METADATA_LOCK);
+    Assert.assertTrue(zkl.lock());
+
+    ZooKeeperLocking zk2 = new ZooKeeperLocking(
+        "D:/carbondata/examples/target/store/default/t3/Metadata", LockUsage.METADATA_LOCK);
+    Assert.assertTrue(!zk2.lock());
+
+    Assert.assertTrue(zkl.unlock());
+    Assert.assertTrue(zk2.lock());
+    Assert.assertTrue(zk2.unlock());
+  }
+
+  /**
+   * For finding the free port available.
+   *
+   * @return
+   */
+  private static int findFreePort() {
+    ServerSocket socket = null;
+    try {
+      socket = new ServerSocket(0);
+      socket.setReuseAddress(true);
+      int port = socket.getLocalPort();
+      try {
+        socket.close();
+      } catch (IOException e) {
+        // Ignore IOException on close()
+      }
+      return port;
+    } catch (Exception e) {
+      // Ignore
+    } finally {
+      if (socket != null) {
+        try {
+          socket.close();
+        } catch (IOException e) {
+          throw new RuntimeException(e);
+        }
+      }
+    }
+    return 2181;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
----------------------------------------------------------------------
diff --git a/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java b/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
index 641d18a..d84c979 100644
--- a/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
+++ b/processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java
@@ -11,11 +11,8 @@ import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
 import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
-import org.carbondata.processing.store.colgroup.ColumnDataHolder;
-import org.carbondata.processing.store.colgroup.DataHolder;
-import org.carbondata.processing.store.colgroup.ColGroupBlockStorage;
-import org.carbondata.processing.store.colgroup.ColGroupDataHolder;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
+
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;


[10/56] [abbrv] incubator-carbondata git commit: Update SQL planning in carbon-spark (#682)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
index 56deb8a..829c487 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
@@ -15,38 +15,27 @@
  * limitations under the License.
  */
 
-
 package org.apache.spark.sql.hive
 
-import scala.math.BigInt.int2bigInt
+import scala.collection.JavaConverters._
 
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.UnresolvedRelation
-import org.apache.spark.sql.catalyst.expressions._
-import org.apache.spark.sql.catalyst.expressions.aggregate.AggregateExpression
-import org.apache.spark.sql.catalyst.planning.{ExtractEquiJoinKeys, PhysicalOperation, QueryPlanner}
-import org.apache.spark.sql.catalyst.plans.Inner
-import org.apache.spark.sql.catalyst.plans.logical.{BroadcastHint, Limit, LogicalPlan, Sort}
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions
+import org.apache.spark.sql.catalyst.expressions.{AttributeSet, _}
+import org.apache.spark.sql.catalyst.planning.{PhysicalOperation, QueryPlanner}
+import org.apache.spark.sql.catalyst.plans.logical.{Filter => LogicalFilter, LogicalPlan}
 import org.apache.spark.sql.execution.{DescribeCommand => RunnableDescribeCommand, ExecutedCommand, Filter, Project, SparkPlan}
-import org.apache.spark.sql.execution.aggregate.Utils
 import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.datasources.{DescribeCommand => LogicalDescribeCommand, LogicalRelation}
-import org.apache.spark.sql.execution.joins.{BroadCastFilterPushJoin, BuildLeft, BuildRight}
 import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, DropTable, HiveNativeCommand}
+import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
 
 import org.carbondata.common.logging.LogServiceFactory
+import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 
-object CarbonHiveSyntax {
-
-  @transient
-  protected val sqlParser = new CarbonSqlParser
-
-  def parse(sqlText: String): LogicalPlan = {
-    sqlParser.parse(sqlText)
-  }
-}
 
 class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
 
@@ -55,342 +44,179 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
   val LOGGER = LogServiceFactory.getLogService("CarbonStrategies")
 
   def getStrategies: Seq[Strategy] = {
-    val total = sqlContext.planner.strategies :+ CarbonTableScans :+ DDLStrategies
+    val total = sqlContext.planner.strategies :+ CarbonTableScan
     total
   }
 
   /**
-   * Carbon strategies for Carbon cube scanning
+   * Carbon strategies for performing late materizlization (decoding dictionary key
+   * as late as possbile)
    */
-  private[sql] object CarbonTableScans extends Strategy {
-
-    def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
-      case PhysicalOperation(projectList, predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)) =>
-        carbonScan(projectList,
-          predicates,
-          carbonRelation.carbonRelation,
-          None,
-          None,
-          None,
-          isGroupByPresent = false,
-          detailQuery = true) :: Nil
-
-      case Limit(IntegerLiteral(limit),
-      Sort(order, _,
-      p@CarbonAggregation(groupingExpressions,
-      aggregateExpressions,
-      PhysicalOperation(
-      projectList,
-      predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))))) =>
-        val aggPlan = handleAggregation(plan, p, projectList, predicates, carbonRelation,
-          groupingExpressions, aggregateExpressions)
-        org.apache.spark.sql.execution.TakeOrderedAndProject(limit,
-          order,
-          None,
-          aggPlan.head) :: Nil
-
-      case Limit(IntegerLiteral(limit), p@CarbonAggregation(
-      groupingExpressions,
-      aggregateExpressions,
-      PhysicalOperation(projectList, predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)))) =>
-        val aggPlan = handleAggregation(plan, p, projectList, predicates, carbonRelation,
-          groupingExpressions, aggregateExpressions)
-        org.apache.spark.sql.execution.Limit(limit, aggPlan.head) :: Nil
-
-      case CarbonAggregation(
-      groupingExpressions,
-      aggregateExpressions,
-      PhysicalOperation(projectList, predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))) =>
-        handleAggregation(plan, plan, projectList, predicates, carbonRelation,
-          groupingExpressions, aggregateExpressions)
-
-      case Limit(IntegerLiteral(limit),
-      PhysicalOperation(projectList, predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))) =>
-        val (_, _, _, _, groupExprs, substitutesortExprs, limitExpr) = extractPlan(plan)
-        val s = carbonScan(projectList, predicates, carbonRelation.carbonRelation, groupExprs,
-          substitutesortExprs, limitExpr, isGroupByPresent = false, detailQuery = true)
-        org.apache.spark.sql.execution.Limit(limit, s) :: Nil
-
-      case Limit(IntegerLiteral(limit),
-      Sort(order, _,
-      PhysicalOperation(projectList, predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)))) =>
-        val (_, _, _, _, groupExprs, substitutesortExprs, limitExpr) = extractPlan(plan)
-        val s = carbonScan(projectList, predicates, carbonRelation.carbonRelation, groupExprs,
-          substitutesortExprs, limitExpr, isGroupByPresent = false, detailQuery = true)
-        org.apache.spark.sql.execution.TakeOrderedAndProject(limit,
-          order,
-          None,
-          s) :: Nil
-
-      case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition,
-      PhysicalOperation(projectList, predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)), right)
-        if canPushDownJoin(right, condition) =>
-        LOGGER.info(s"pushing down for ExtractEquiJoinKeys:right")
-        val carbon = carbonScan(projectList,
-          predicates,
-          carbonRelation.carbonRelation,
-          None,
-          None,
-          None,
-          isGroupByPresent = false,
-          detailQuery = true)
-        val pushedDownJoin = BroadCastFilterPushJoin(
-          leftKeys: Seq[Expression],
-          rightKeys: Seq[Expression],
-          BuildRight,
-          carbon,
-          planLater(right),
-          condition)
-
-        condition.map(Filter(_, pushedDownJoin)).getOrElse(pushedDownJoin) :: Nil
-
-      case ExtractEquiJoinKeys(Inner, leftKeys, rightKeys, condition, left,
-      PhysicalOperation(projectList, predicates,
-      l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)))
-        if canPushDownJoin(left, condition) =>
-        LOGGER.info(s"pushing down for ExtractEquiJoinKeys:left")
-        val carbon = carbonScan(projectList,
-          predicates,
-          carbonRelation.carbonRelation,
-          None,
-          None,
-          None,
-          isGroupByPresent = false,
-          detailQuery = true)
-
-        val pushedDownJoin = BroadCastFilterPushJoin(
-          leftKeys: Seq[Expression],
-          rightKeys: Seq[Expression],
-          BuildLeft,
-          planLater(left),
-          carbon,
-          condition)
-        condition.map(Filter(_, pushedDownJoin)).getOrElse(pushedDownJoin) :: Nil
-
-      case _ => Nil
-    }
-
-    def handleAggregation(plan: LogicalPlan,
-        aggPlan: LogicalPlan,
-        projectList: Seq[NamedExpression],
-        predicates: Seq[Expression],
-        carbonRelation: CarbonDatasourceRelation,
-        groupingExpressions: Seq[Expression],
-        namedGroupingAttributes: Seq[NamedExpression]):
-    Seq[SparkPlan] = {
-      val (_, _, _, _, groupExprs, substitutesortExprs, limitExpr) = extractPlan(plan)
-
-      val s =
-        try {
-          carbonScan(projectList, predicates, carbonRelation.carbonRelation,
-            Some(namedGroupingAttributes), substitutesortExprs,
-            limitExpr, groupingExpressions.nonEmpty)
-        } catch {
-          case e: Exception => null
-        }
-
-      if (s != null) {
-        aggregatePlan(groupingExpressions, namedGroupingAttributes, s)
-
-      } else {
-        (aggPlan, true) match {
-          case CarbonAggregation(
-          groupingExpressions,
-          namedGroupingAttributes,
-          PhysicalOperation(projectList, predicates,
-          l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _))) =>
-            val (_, _, _, _, groupExprs, substitutesortExprs, limitExpr) = extractPlan(plan)
-
-
-            val s = carbonScan(projectList, predicates, carbonRelation.carbonRelation,
-              Some(namedGroupingAttributes), substitutesortExprs, limitExpr,
-              groupingExpressions.nonEmpty, detailQuery = true)
-
-            aggregatePlan(groupingExpressions, namedGroupingAttributes, s)
-        }
+  private[sql] object CarbonTableScan extends Strategy {
+
+    def apply(plan: LogicalPlan): Seq[SparkPlan] = {
+      plan match {
+        case PhysicalOperation(projectList, predicates,
+        l@LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)) =>
+          if (isStarQuery(plan)) {
+            carbonRawScanForStarQuery(projectList, predicates, carbonRelation, l)(sqlContext) :: Nil
+          } else {
+            carbonRawScan(projectList,
+              predicates,
+              carbonRelation,
+              l,
+              None,
+              detailQuery = true,
+              useBinaryAggregation = false)(sqlContext)._1 :: Nil
+          }
+        case CarbonDictionaryCatalystDecoder(relations, profile, aliasMap, _, child) =>
+          CarbonDictionaryDecoder(relations,
+            profile,
+            aliasMap,
+            planLater(child))(sqlContext) :: Nil
+        case _ =>
+          Nil
       }
     }
 
-    // TODO: It is duplicated code from spark. Need to find a way
-    private def aggregatePlan(groupingExpressions: Seq[Expression],
-        resultExpressions: Seq[NamedExpression],
-        child: SparkPlan) = {
-      // A single aggregate expression might appear multiple times in resultExpressions.
-      // In order to avoid evaluating an individual aggregate function multiple times, we'll
-      // build a set of the distinct aggregate expressions and build a function which can
-      // be used to re-write expressions so that they reference the single copy of the
-      // aggregate function which actually gets computed.
-      val aggregateExpressions = resultExpressions.flatMap { expr =>
-        expr.collect {
-          case agg: AggregateExpression => agg
-        }
-      }.distinct
-      // For those distinct aggregate expressions, we create a map from the
-      // aggregate function to the corresponding attribute of the function.
-      val aggregateFunctionToAttribute = aggregateExpressions.map { agg =>
-        val aggregateFunction = agg.aggregateFunction
-        val attribute = Alias(aggregateFunction, aggregateFunction.toString)().toAttribute
-        (aggregateFunction, agg.isDistinct) -> attribute
-      }.toMap
-
-      val (functionsWithDistinct, functionsWithoutDistinct) =
-        aggregateExpressions.partition(_.isDistinct)
-      if (functionsWithDistinct.map(_.aggregateFunction.children).distinct.length > 1) {
-        // This is a sanity check. We should not reach here when we have multiple distinct
-        // column sets. Our MultipleDistinctRewriter should take care this case.
-        sys.error("You hit a query analyzer bug. Please report your query to " +
-                  "Spark user mailing list.")
-      }
-
-      val namedGroupingExpressions = groupingExpressions.map {
-        case ne: NamedExpression => ne -> ne
-        // If the expression is not a NamedExpressions, we add an alias.
-        // So, when we generate the result of the operator, the Aggregate Operator
-        // can directly get the Seq of attributes representing the grouping expressions.
-        case other =>
-          val withAlias = Alias(other, other.toString)()
-          other -> withAlias
-      }
-      val groupExpressionMap = namedGroupingExpressions.toMap
-
-      // The original `resultExpressions` are a set of expressions which may reference
-      // aggregate expressions, grouping column values, and constants. When aggregate operator
-      // emits output rows, we will use `resultExpressions` to generate an output projection
-      // which takes the grouping columns and final aggregate result buffer as input.
-      // Thus, we must re-write the result expressions so that their attributes match up with
-      // the attributes of the final result projection's input row:
-      val rewrittenResultExpressions = resultExpressions.map { expr =>
-        expr.transformDown {
-          case AggregateExpression(aggregateFunction, _, isDistinct) =>
-            // The final aggregation buffer's attributes will be `finalAggregationAttributes`,
-            // so replace each aggregate expression by its corresponding attribute in the set:
-            aggregateFunctionToAttribute(aggregateFunction, isDistinct)
-          case expression =>
-            // Since we're using `namedGroupingAttributes` to extract the grouping key
-            // columns, we need to replace grouping key expressions with their corresponding
-            // attributes. We do not rely on the equality check at here since attributes may
-            // differ cosmetically. Instead, we use semanticEquals.
-            groupExpressionMap.collectFirst {
-              case (expr, ne) if expr semanticEquals expression => ne.toAttribute
-            }.getOrElse(expression)
-        }.asInstanceOf[NamedExpression]
+    /**
+     * Create carbon scan
+     */
+    private def carbonRawScan(projectList: Seq[NamedExpression],
+      predicates: Seq[Expression],
+      relation: CarbonDatasourceRelation,
+      logicalRelation: LogicalRelation,
+      groupExprs: Option[Seq[Expression]],
+      detailQuery: Boolean,
+      useBinaryAggregation: Boolean)(sc: SQLContext): (SparkPlan, Boolean) = {
+
+      val tableName: String =
+        relation.carbonRelation.metaData.carbonTable.getFactTableName.toLowerCase
+      // Check out any expressions are there in project list. if they are present then we need to
+      // decode them as well.
+      val projectSet = AttributeSet(projectList.flatMap(_.references))
+      val scan = CarbonScan(projectSet.toSeq,
+        relation.carbonRelation,
+        predicates,
+        groupExprs,
+        useBinaryAggregation)(sqlContext)
+      val dimAggrsPresence: Boolean = scan.buildCarbonPlan.getDimAggregatorInfos.size() > 0
+      projectList.map {
+        case attr: AttributeReference =>
+        case Alias(attr: AttributeReference, _) =>
+        case others =>
+          others.references
+              .map(f => scan.attributesNeedToDecode.add(f.asInstanceOf[AttributeReference]))
       }
-
-      val aggregateOperator =
-        if (aggregateExpressions.map(_.aggregateFunction).exists(!_.supportsPartial)) {
-          if (functionsWithDistinct.nonEmpty) {
-            sys.error("Distinct columns cannot exist in Aggregate operator containing " +
-                      "aggregate functions which don't support partial aggregation.")
+      if (!detailQuery) {
+        if (scan.attributesNeedToDecode.size > 0) {
+          val decoder = getCarbonDecoder(logicalRelation,
+            sc,
+            tableName,
+            scan.attributesNeedToDecode.asScala.toSeq,
+            scan)
+          if (scan.unprocessedExprs.nonEmpty) {
+            val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
+            (Project(projectList, filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)), true)
           } else {
-            Utils.planAggregateWithoutPartial(
-              namedGroupingExpressions.map(_._2),
-              aggregateExpressions,
-              aggregateFunctionToAttribute,
-              rewrittenResultExpressions,
-              child)
+            (Project(projectList, decoder), true)
           }
-        } else if (functionsWithDistinct.isEmpty) {
-          Utils.planAggregateWithoutDistinct(
-            namedGroupingExpressions.map(_._2),
-            aggregateExpressions,
-            aggregateFunctionToAttribute,
-            rewrittenResultExpressions,
-            child)
         } else {
-          Utils.planAggregateWithOneDistinct(
-            namedGroupingExpressions.map(_._2),
-            functionsWithDistinct,
-            functionsWithoutDistinct,
-            aggregateFunctionToAttribute,
-            rewrittenResultExpressions,
-            child)
+          (scan, dimAggrsPresence)
+        }
+      } else {
+        if (scan.attributesNeedToDecode.size() > 0) {
+          val decoder = getCarbonDecoder(logicalRelation,
+            sc,
+            tableName,
+            scan.attributesNeedToDecode.asScala.toSeq,
+            scan)
+          if (scan.unprocessedExprs.nonEmpty) {
+            val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
+            (Project(projectList, filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)), true)
+          } else {
+            (Project(projectList, decoder), true)
+          }
+        } else {
+          (Project(projectList, scan), dimAggrsPresence)
         }
-
-      aggregateOperator
-    }
-
-    private def canPushDownJoin(otherRDDPlan: LogicalPlan,
-        joinCondition: Option[Expression]): Boolean = {
-      val pushdowmJoinEnabled = sqlContext.sparkContext.conf
-        .getBoolean("spark.carbon.pushdown.join.as.filter", defaultValue = true)
-
-      if (!pushdowmJoinEnabled) {
-        return false
-      }
-
-      otherRDDPlan match {
-        case BroadcastHint(p) => true
-        case p if sqlContext.conf.autoBroadcastJoinThreshold > 0 &&
-                  p.statistics.sizeInBytes <= sqlContext.conf.autoBroadcastJoinThreshold =>
-          LOGGER.info("canPushDownJoin statistics:" + p.statistics.sizeInBytes)
-          true
-        case _ => false
       }
     }
 
     /**
-     * Create carbon scan
+     * Create carbon scan for star query
      */
-    private def carbonScan(projectList: Seq[NamedExpression],
-        predicates: Seq[Expression],
-        relation: CarbonRelation,
-        groupExprs: Option[Seq[Expression]],
-        substitutesortExprs: Option[Seq[SortOrder]],
-        limitExpr: Option[Expression],
-        isGroupByPresent: Boolean,
-        detailQuery: Boolean = false) = {
+    private def carbonRawScanForStarQuery(projectList: Seq[NamedExpression],
+      predicates: Seq[Expression],
+      relation: CarbonDatasourceRelation,
+      logicalRelation: LogicalRelation)(sc: SQLContext): SparkPlan = {
+
+      val tableName: String =
+        relation.carbonRelation.metaData.carbonTable.getFactTableName.toLowerCase
+      // Check out any expressions are there in project list. if they are present then we need to
+      // decode them as well.
+      val projectExprsNeedToDecode = new java.util.HashSet[Attribute]()
+      val scan = CarbonScan(projectList.map(_.toAttribute),
+        relation.carbonRelation,
+        predicates,
+        None,
+        useBinaryAggregator = false)(sqlContext)
+      projectExprsNeedToDecode.addAll(scan.attributesNeedToDecode)
+      if (projectExprsNeedToDecode.size() > 0) {
+        val decoder = getCarbonDecoder(logicalRelation,
+          sc,
+          tableName,
+          projectExprsNeedToDecode.asScala.toSeq,
+          scan)
+        if (scan.unprocessedExprs.nonEmpty) {
+          val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
+          filterCondToAdd.map(Filter(_, decoder)).getOrElse(decoder)
+        } else {
+          decoder
+        }
+      } else {
+        scan
+      }
+    }
 
-      if (!detailQuery) {
-        val projectSet = AttributeSet(projectList.flatMap(_.references))
-        CarbonTableScan(
-          projectSet.toSeq,
-          relation,
-          predicates,
-          groupExprs,
-          substitutesortExprs,
-          limitExpr,
-          isGroupByPresent,
-          detailQuery)(sqlContext)
+    def getCarbonDecoder(logicalRelation: LogicalRelation,
+      sc: SQLContext,
+      tableName: String,
+      projectExprsNeedToDecode: Seq[Attribute],
+      scan: CarbonScan): CarbonDictionaryDecoder = {
+      val relation = CarbonDecoderRelation(logicalRelation.attributeMap,
+        logicalRelation.relation.asInstanceOf[CarbonDatasourceRelation])
+      val attrs = projectExprsNeedToDecode.map { attr =>
+        val newAttr = AttributeReference(attr.name,
+          attr.dataType,
+          attr.nullable,
+          attr.metadata)(attr.exprId, Seq(tableName))
+        relation.addAttribute(newAttr)
+        newAttr
       }
-      else {
-        val projectSet = AttributeSet(projectList.flatMap(_.references))
-        Project(projectList,
-          CarbonTableScan(projectSet.toSeq,
-            relation,
-            predicates,
-            groupExprs,
-            substitutesortExprs,
-            limitExpr,
-            isGroupByPresent,
-            detailQuery)(sqlContext))
+      CarbonDictionaryDecoder(Seq(relation), IncludeProfile(attrs),
+        CarbonAliasDecoderRelation(), scan)(sc)
+    }
 
+    private def isStarQuery(plan: LogicalPlan) = {
+      plan match {
+        case LogicalFilter(condition,
+        LogicalRelation(carbonRelation: CarbonDatasourceRelation, _)) => true
+        case LogicalRelation(carbonRelation: CarbonDatasourceRelation, _) => true
+        case _ => false
       }
     }
 
-    private def extractPlan(plan: LogicalPlan) = {
-      val (a, b, c, aliases, groupExprs, sortExprs, limitExpr) =
-        PhysicalOperation1.collectProjectsAndFilters(plan)
-      val substitutesortExprs = sortExprs match {
-        case Some(sort) =>
-          Some(sort.map {
-            case SortOrder(a: Alias, direction) =>
-              val ref = aliases.getOrElse(a.toAttribute, a) match {
-                case Alias(reference, name) => reference
-                case others => others
-              }
-              SortOrder(ref, direction)
-            case others => others
-          })
-        case others => others
+    private def isGroupByPresentOnMeasures(groupingExpressions: Seq[Expression],
+      carbonTable: CarbonTable): Boolean = {
+      groupingExpressions.map { g =>
+        g.collect {
+          case attr: AttributeReference
+            if carbonTable.getMeasureByName(carbonTable.getFactTableName, attr.name) != null =>
+            return true
+        }
       }
-      (a, b, c, aliases, groupExprs, substitutesortExprs, limitExpr)
+      false
     }
   }
 
@@ -406,7 +232,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         ExecutedCommand(ShowAllTablesDetail(schemaName, plan.output)) :: Nil
       case DropTable(tableName, ifNotExists)
         if CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .tableExists(TableIdentifier(tableName.toLowerCase))(sqlContext) =>
+            .tableExists(TableIdentifier(tableName.toLowerCase))(sqlContext) =>
         ExecutedCommand(DropCubeCommand(ifNotExists, None, tableName.toLowerCase)) :: Nil
       case ShowAggregateTablesCommand(schemaName) =>
         ExecutedCommand(ShowAggregateTables(schemaName, plan.output)) :: Nil
@@ -415,7 +241,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
       case LoadCube(schemaNameOp, cubeName, factPathFromUser, dimFilesPath,
       partionValues, isOverwriteExist, inputSqlString) =>
         val isCarbonTable = CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .tableExists(TableIdentifier(cubeName, schemaNameOp))(sqlContext)
+            .tableExists(TableIdentifier(cubeName, schemaNameOp))(sqlContext)
         if (isCarbonTable || partionValues.nonEmpty) {
           ExecutedCommand(LoadCube(schemaNameOp, cubeName, factPathFromUser,
             dimFilesPath, partionValues, isOverwriteExist, inputSqlString)) :: Nil
@@ -433,15 +259,14 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         }
       case DescribeFormattedCommand(sql, tblIdentifier) =>
         val isCube = CarbonEnv.getInstance(sqlContext).carbonCatalog
-          .tableExists(tblIdentifier)(sqlContext)
+            .tableExists(tblIdentifier)(sqlContext)
         if (isCube) {
           val describe =
             LogicalDescribeCommand(UnresolvedRelation(tblIdentifier, None), isExtended = false)
           val resolvedTable = sqlContext.executePlan(describe.table).analyzed
           val resultPlan = sqlContext.executePlan(resolvedTable).executedPlan
           ExecutedCommand(DescribeCommandFormatted(resultPlan, plan.output, tblIdentifier)) :: Nil
-        }
-        else {
+        } else {
           ExecutedCommand(DescribeNativeCommand(sql, plan.output)) :: Nil
         }
       case describe@LogicalDescribeCommand(table, isExtended) =>
@@ -461,3 +286,13 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
   }
 
 }
+
+object CarbonHiveSyntax {
+
+  @transient
+  protected val sqlParser = new CarbonSqlParser
+
+  def parse(sqlText: String): LogicalPlan = {
+    sqlParser.parse(sqlText)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala
deleted file mode 100644
index 58c02ff..0000000
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategy.scala
+++ /dev/null
@@ -1,54 +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.hive
-
-import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-
-import org.carbondata.spark.exception.MalformedCarbonCommandException
-
-private[sql] object CarbonStrategy {
-  def getStrategy(context: SQLContext): Seq[Strategy] = {
-    val carbonStrategy = new CarbonStrategies(context)
-    if (context.conf.asInstanceOf[CarbonSQLConf].pushComputation) {
-      Seq(carbonStrategy.CarbonTableScans, carbonStrategy.DDLStrategies)
-    } else {
-      // TODO: need to remove duplicate code in strategies.
-      Seq(new CarbonRawStrategies(context).CarbonRawTableScans, carbonStrategy.DDLStrategies)
-    }
-  }
-}
-
-private[spark] class CarbonSQLDialect(context: HiveContext) extends HiveQLDialect(context) {
-
-  @transient
-  protected val sqlParser = new CarbonSqlParser
-
-  override def parse(sqlText: String): LogicalPlan = {
-
-    try {
-      sqlParser.parse(sqlText)
-    } catch {
-      // MalformedCarbonCommandException need to throw directly
-      // because hive can no parse carbon command
-      case ce: MalformedCarbonCommandException =>
-        throw ce
-      case _ => super.parse(sqlText)
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonQueryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonQueryRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonQueryRDD.scala
deleted file mode 100644
index 60c6a78..0000000
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonQueryRDD.scala
+++ /dev/null
@@ -1,241 +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.carbondata.spark.rdd
-
-import java.util
-
-import scala.collection.JavaConverters._
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.hadoop.mapreduce.Job
-import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
-import org.apache.spark.rdd.RDD
-
-import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.cache.dictionary.Dictionary
-import org.carbondata.core.carbon.datastore.block.TableBlockInfo
-import org.carbondata.core.iterator.CarbonIterator
-import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
-import org.carbondata.query.carbon.executor.QueryExecutorFactory
-import org.carbondata.query.carbon.model.QueryModel
-import org.carbondata.query.carbon.result.RowResult
-import org.carbondata.query.expression.Expression
-import org.carbondata.query.filter.resolver.FilterResolverIntf
-import org.carbondata.spark.KeyVal
-import org.carbondata.spark.load.CarbonLoaderUtil
-import org.carbondata.spark.util.QueryPlanUtil
-
-class CarbonSparkPartition(rddId: Int, val idx: Int,
-  val locations: Array[String],
-  val tableBlockInfos: util.List[TableBlockInfo])
-  extends Partition {
-
-  override val index: Int = idx
-
-  // val serializableHadoopSplit = new SerializableWritable[Array[String]](locations)
-  override def hashCode(): Int = {
-    41 * (41 + rddId) + idx
-  }
-}
-
-
- /**
-  * This RDD is used to perform query.
-  */
-class CarbonQueryRDD[K, V](
-  sc: SparkContext,
-  queryModel: QueryModel,
-  filterExpression: Expression,
-  keyClass: KeyVal[K, V],
-  @transient conf: Configuration,
-  cubeCreationTime: Long,
-  schemaLastUpdatedTime: Long,
-  baseStoreLocation: String)
-  extends RDD[(K, V)](sc, Nil) with Logging {
-
-  val defaultParallelism = sc.defaultParallelism
-
-  override def getPartitions: Array[Partition] = {
-    val startTime = System.currentTimeMillis()
-    val (carbonInputFormat: CarbonInputFormat[RowResult], job: Job) =
-      QueryPlanUtil.createCarbonInputFormat(queryModel.getAbsoluteTableIdentifier)
-
-    val result = new util.ArrayList[Partition](defaultParallelism)
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    // set filter resolver tree
-    try {
-      var filterResolver = carbonInputFormat
-        .getResolvedFilter(job.getConfiguration, filterExpression)
-
-      CarbonInputFormat.setFilterPredicates(job.getConfiguration, filterResolver)
-      queryModel.setFilterExpressionResolverTree(filterResolver)
-    }
-    catch {
-      case e: Exception =>
-        LOGGER.error(e)
-        sys.error("Exception occurred in query execution :: " + e.getMessage)
-    }
-    // get splits
-    val splits = carbonInputFormat.getSplits(job)
-    if (!splits.isEmpty) {
-      val carbonInputSplits = splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
-
-      val blockList = carbonInputSplits.map(inputSplit =>
-        new TableBlockInfo(inputSplit.getPath.toString,
-          inputSplit.getStart, inputSplit.getSegmentId,
-          inputSplit.getLocations, inputSplit.getLength
-        )
-      )
-      if (blockList.nonEmpty) {
-        // group blocks to nodes, tasks
-        val nodeBlockMapping =
-          CarbonLoaderUtil.nodeBlockTaskMapping(blockList.asJava, -1, defaultParallelism)
-
-        var i = 0
-        // Create Spark Partition for each task and assign blocks
-        nodeBlockMapping.asScala.foreach { entry =>
-          entry._2.asScala.foreach { blocksPerTask =>
-            if (blocksPerTask.size() != 0) {
-              result.add(new CarbonSparkPartition(id, i, Seq(entry._1).toArray, blocksPerTask))
-              i += 1
-            }
-          }
-        }
-        val noOfBlocks = blockList.size
-        val noOfNodes = nodeBlockMapping.size
-        val noOfTasks = result.size()
-        logInfo(s"Identified  no.of.Blocks: $noOfBlocks,"
-          + s"parallelism: $defaultParallelism , " +
-          s"no.of.nodes: $noOfNodes, no.of.tasks: $noOfTasks"
-        )
-        logInfo("Time taken to identify Blocks to scan : " +
-          (System.currentTimeMillis() - startTime)
-        )
-        result.asScala.foreach { r =>
-          val cp = r.asInstanceOf[CarbonSparkPartition]
-          logInfo(s"Node : " + cp.locations.toSeq.mkString(",")
-            + ", No.Of Blocks : " + cp.tableBlockInfos.size()
-          )
-        }
-      } else {
-        logInfo("No blocks identified to scan")
-        val nodesPerBlock = new util.ArrayList[TableBlockInfo]()
-        result.add(new CarbonSparkPartition(id, 0, Seq("").toArray, nodesPerBlock))
-      }
-    }
-    else {
-      logInfo("No valid segments found to scan")
-      val nodesPerBlock = new util.ArrayList[TableBlockInfo]()
-      result.add(new CarbonSparkPartition(id, 0, Seq("").toArray, nodesPerBlock))
-    }
-    result.toArray(new Array[Partition](result.size()))
-  }
-
-
-  override def compute(thepartition: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
-    val iter = new Iterator[(K, V)] {
-      var rowIterator: CarbonIterator[_] = _
-      var queryStartTime: Long = 0
-      try {
-        val carbonSparkPartition = thepartition.asInstanceOf[CarbonSparkPartition]
-        if (!carbonSparkPartition.tableBlockInfos.isEmpty) {
-          queryModel.setQueryId(queryModel.getQueryId + "_" + carbonSparkPartition.idx)
-          // fill table block info
-          queryModel.setTableBlockInfos(carbonSparkPartition.tableBlockInfos)
-          queryStartTime = System.currentTimeMillis
-
-          val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null)
-          logInfo("*************************" + carbonPropertiesFilePath)
-          if (null == carbonPropertiesFilePath) {
-            System.setProperty("carbon.properties.filepath",
-              System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties"
-            )
-          }
-          // execute query
-          rowIterator = QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel)
-            .asInstanceOf[CarbonIterator[RowResult]]
-        }
-        // TODOi
-        // : CarbonQueryUtil.isQuickFilter quick filter from dictionary needs to support
-      } catch {
-        case e: Throwable =>
-          clearDictionaryCache(queryModel.getColumnToDictionaryMapping)
-          LOGGER.error(e)
-          // updateCubeAndLevelCacheStatus(levelCacheKeys)
-          if (null != e.getMessage) {
-            sys.error("Exception occurred in query execution :: " + e.getMessage)
-          } else {
-            sys.error("Exception occurred in query execution.Please check logs.")
-          }
-      }
-
-      var havePair = false
-      var finished = false
-      var recordCount = 0
-
-      override def hasNext: Boolean = {
-        if (!finished && !havePair) {
-          finished = (null == rowIterator) || (!rowIterator.hasNext)
-          havePair = !finished
-        }
-        if (finished) {
-          clearDictionaryCache(queryModel.getColumnToDictionaryMapping)
-        }
-        !finished
-      }
-
-      override def next(): (K, V) = {
-        if (!hasNext) {
-          throw new java.util.NoSuchElementException("End of stream")
-        }
-        havePair = false
-        val row = rowIterator.next()
-        val key = row.asInstanceOf[RowResult].getKey()
-        val value = row.asInstanceOf[RowResult].getValue()
-        recordCount += 1
-        if (queryModel.getLimit != -1 && recordCount >= queryModel.getLimit) {
-          clearDictionaryCache(queryModel.getColumnToDictionaryMapping)
-        }
-        keyClass.getKey(key, value)
-      }
-
-      def clearDictionaryCache(columnToDictionaryMap: java.util.Map[String, Dictionary]) = {
-        if (null != columnToDictionaryMap) {
-          org.carbondata.spark.util.CarbonQueryUtil
-            .clearColumnDictionaryCache(columnToDictionaryMap)
-        }
-      }
-
-      logInfo("*************************** Total Time Taken to execute the query in Carbon Side: " +
-        (System.currentTimeMillis - queryStartTime)
-      )
-    }
-    iter
-  }
-
-
-   /**
-    * Get the preferred locations where to launch this task.
-    */
-  override def getPreferredLocations(partition: Partition): Seq[String] = {
-    val theSplit = partition.asInstanceOf[CarbonSparkPartition]
-    theSplit.locations.filter(_ != "localhost")
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala
deleted file mode 100644
index 5993677..0000000
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonRawQueryRDD.scala
+++ /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.carbondata.spark.rdd
-
-import org.apache.hadoop.conf.Configuration
-import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
-
-import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.iterator.CarbonIterator
-import org.carbondata.query.carbon.executor.QueryExecutorFactory
-import org.carbondata.query.carbon.model.QueryModel
-import org.carbondata.query.carbon.result.BatchRawResult
-import org.carbondata.query.carbon.result.iterator.ChunkRawRowIterartor
-import org.carbondata.query.expression.Expression
-import org.carbondata.spark.{RawKey, RawKeyVal}
-
-
-/**
- * This RDD is used to perform query with raw data, it means it doesn't convert dictionary values
- * to actual data.
- *
- * @param sc
- * @param queryModel
- * @param filterExpression
- * @param keyClass
- * @param conf
- * @param cubeCreationTime
- * @param schemaLastUpdatedTime
- * @param baseStoreLocation
- * @tparam K
- * @tparam V
- */
-class CarbonRawQueryRDD[K, V](
-    sc: SparkContext,
-    queryModel: QueryModel,
-    filterExpression: Expression,
-    keyClass: RawKey[K, V],
-    @transient conf: Configuration,
-    cubeCreationTime: Long,
-    schemaLastUpdatedTime: Long,
-    baseStoreLocation: String)
-  extends CarbonQueryRDD[K, V](sc,
-    queryModel,
-    filterExpression,
-    null,
-    conf,
-    cubeCreationTime,
-    schemaLastUpdatedTime,
-    baseStoreLocation) with Logging {
-
-
-  override def compute(thepartition: Partition, context: TaskContext): Iterator[(K, V)] = {
-    val LOGGER = LogServiceFactory.getLogService(this.getClass().getName());
-    val iter = new Iterator[(K, V)] {
-      var rowIterator: CarbonIterator[Array[Any]] = _
-      var queryStartTime: Long = 0
-      try {
-        val carbonSparkPartition = thepartition.asInstanceOf[CarbonSparkPartition]
-        if(!carbonSparkPartition.tableBlockInfos.isEmpty) {
-          queryModel.setQueryId(queryModel.getQueryId() + "_" + carbonSparkPartition.idx)
-          // fill table block info
-          queryModel.setTableBlockInfos(carbonSparkPartition.tableBlockInfos)
-          queryStartTime = System.currentTimeMillis
-
-          val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null)
-          logInfo("*************************" + carbonPropertiesFilePath)
-          if (null == carbonPropertiesFilePath) {
-            System.setProperty("carbon.properties.filepath",
-              System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties");
-          }
-          // execute query
-          rowIterator = new ChunkRawRowIterartor(
-            QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel)
-            .asInstanceOf[CarbonIterator[BatchRawResult]]).asInstanceOf[CarbonIterator[Array[Any]]]
-        }
-      } catch {
-        case e: Exception =>
-          LOGGER.error(e)
-          // updateCubeAndLevelCacheStatus(levelCacheKeys)
-          if (null != e.getMessage) {
-            sys.error("Exception occurred in query execution :: " + e.getMessage)
-          } else {
-            sys.error("Exception occurred in query execution.Please check logs.")
-          }
-      }
-
-      var havePair = false
-      var finished = false
-
-      override def hasNext: Boolean = {
-        if (!finished && !havePair) {
-          finished = (null == rowIterator) || (!rowIterator.hasNext())
-          havePair = !finished
-        }
-        !finished
-      }
-
-      override def next(): (K, V) = {
-        if (!hasNext) {
-          throw new java.util.NoSuchElementException("End of stream")
-        }
-        havePair = false
-        val row = rowIterator.next()
-        keyClass.getKey(row, null)
-      }
-
-      logInfo("*************************** Total Time Taken to execute the query in Carbon Side: " +
-              (System.currentTimeMillis - queryStartTime)
-      )
-    }
-    iter
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
new file mode 100644
index 0000000..a95ae27
--- /dev/null
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -0,0 +1,219 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+
+package org.carbondata.spark.rdd
+
+import java.util
+
+import scala.collection.JavaConverters._
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.mapreduce.Job
+import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+
+import org.carbondata.common.logging.LogServiceFactory
+import org.carbondata.core.carbon.datastore.block.TableBlockInfo
+import org.carbondata.core.iterator.CarbonIterator
+import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
+import org.carbondata.query.carbon.executor.QueryExecutorFactory
+import org.carbondata.query.carbon.model.QueryModel
+import org.carbondata.query.carbon.result.{BatchRawResult, RowResult}
+import org.carbondata.query.carbon.result.iterator.ChunkRawRowIterartor
+import org.carbondata.query.expression.Expression
+import org.carbondata.spark.RawKey
+import org.carbondata.spark.load.CarbonLoaderUtil
+import org.carbondata.spark.util.QueryPlanUtil
+
+class CarbonSparkPartition(rddId: Int, val idx: Int,
+  val locations: Array[String],
+  val tableBlockInfos: util.List[TableBlockInfo])
+  extends Partition {
+
+  override val index: Int = idx
+
+  // val serializableHadoopSplit = new SerializableWritable[Array[String]](locations)
+  override def hashCode(): Int = {
+    41 * (41 + rddId) + idx
+  }
+}
+
+ /**
+  * This RDD is used to perform query on CarbonData file. Before sending tasks to scan
+  * CarbonData file, this RDD will leverage CarbonData's index information to do CarbonData file
+  * level filtering in driver side.
+  */
+class CarbonScanRDD[K, V](
+  sc: SparkContext,
+  queryModel: QueryModel,
+  filterExpression: Expression,
+  keyClass: RawKey[K, V],
+  @transient conf: Configuration,
+  cubeCreationTime: Long,
+  schemaLastUpdatedTime: Long,
+  baseStoreLocation: String)
+  extends RDD[(K, V)](sc, Nil) with Logging {
+
+  val defaultParallelism = sc.defaultParallelism
+
+  override def getPartitions: Array[Partition] = {
+    val startTime = System.currentTimeMillis()
+    val (carbonInputFormat: CarbonInputFormat[RowResult], job: Job) =
+      QueryPlanUtil.createCarbonInputFormat(queryModel.getAbsoluteTableIdentifier)
+
+    val result = new util.ArrayList[Partition](defaultParallelism)
+    val LOGGER = LogServiceFactory.getLogService(this.getClass.getName)
+    // set filter resolver tree
+    try {
+      var filterResolver = carbonInputFormat
+        .getResolvedFilter(job.getConfiguration, filterExpression)
+
+      CarbonInputFormat.setFilterPredicates(job.getConfiguration, filterResolver)
+      queryModel.setFilterExpressionResolverTree(filterResolver)
+    }
+    catch {
+      case e: Exception =>
+        LOGGER.error(e)
+        sys.error("Exception occurred in query execution :: " + e.getMessage)
+    }
+    // get splits
+    val splits = carbonInputFormat.getSplits(job)
+    if (!splits.isEmpty) {
+      val carbonInputSplits = splits.asScala.map(_.asInstanceOf[CarbonInputSplit])
+
+      val blockList = carbonInputSplits.map(inputSplit =>
+        new TableBlockInfo(inputSplit.getPath.toString,
+          inputSplit.getStart, inputSplit.getSegmentId,
+          inputSplit.getLocations, inputSplit.getLength
+        )
+      )
+      if (blockList.nonEmpty) {
+        // group blocks to nodes, tasks
+        val nodeBlockMapping =
+          CarbonLoaderUtil.nodeBlockTaskMapping(blockList.asJava, -1, defaultParallelism)
+
+        var i = 0
+        // Create Spark Partition for each task and assign blocks
+        nodeBlockMapping.asScala.foreach { entry =>
+          entry._2.asScala.foreach { blocksPerTask =>
+            if (blocksPerTask.size() != 0) {
+              result.add(new CarbonSparkPartition(id, i, Seq(entry._1).toArray, blocksPerTask))
+              i += 1
+            }
+          }
+        }
+        val noOfBlocks = blockList.size
+        val noOfNodes = nodeBlockMapping.size
+        val noOfTasks = result.size()
+        logInfo(s"Identified  no.of.Blocks: $noOfBlocks,"
+          + s"parallelism: $defaultParallelism , " +
+          s"no.of.nodes: $noOfNodes, no.of.tasks: $noOfTasks"
+        )
+        logInfo("Time taken to identify Blocks to scan : " +
+          (System.currentTimeMillis() - startTime)
+        )
+        result.asScala.foreach { r =>
+          val cp = r.asInstanceOf[CarbonSparkPartition]
+          logInfo(s"Node : " + cp.locations.toSeq.mkString(",")
+            + ", No.Of Blocks : " + cp.tableBlockInfos.size()
+          )
+        }
+      } else {
+        logInfo("No blocks identified to scan")
+        val nodesPerBlock = new util.ArrayList[TableBlockInfo]()
+        result.add(new CarbonSparkPartition(id, 0, Seq("").toArray, nodesPerBlock))
+      }
+    }
+    else {
+      logInfo("No valid segments found to scan")
+      val nodesPerBlock = new util.ArrayList[TableBlockInfo]()
+      result.add(new CarbonSparkPartition(id, 0, Seq("").toArray, nodesPerBlock))
+    }
+    result.toArray(new Array[Partition](result.size()))
+  }
+
+   override def compute(thepartition: Partition, context: TaskContext): Iterator[(K, V)] = {
+     val LOGGER = LogServiceFactory.getLogService(this.getClass().getName());
+     val iter = new Iterator[(K, V)] {
+       var rowIterator: CarbonIterator[Array[Any]] = _
+       var queryStartTime: Long = 0
+       try {
+         val carbonSparkPartition = thepartition.asInstanceOf[CarbonSparkPartition]
+         if(!carbonSparkPartition.tableBlockInfos.isEmpty) {
+           queryModel.setQueryId(queryModel.getQueryId() + "_" + carbonSparkPartition.idx)
+           // fill table block info
+           queryModel.setTableBlockInfos(carbonSparkPartition.tableBlockInfos)
+           queryStartTime = System.currentTimeMillis
+
+           val carbonPropertiesFilePath = System.getProperty("carbon.properties.filepath", null)
+           logInfo("*************************" + carbonPropertiesFilePath)
+           if (null == carbonPropertiesFilePath) {
+             System.setProperty("carbon.properties.filepath",
+               System.getProperty("user.dir") + '/' + "conf" + '/' + "carbon.properties");
+           }
+           // execute query
+           rowIterator = new ChunkRawRowIterartor(
+             QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel)
+                 .asInstanceOf[CarbonIterator[BatchRawResult]])
+                 .asInstanceOf[CarbonIterator[Array[Any]]]
+         }
+       } catch {
+         case e: Exception =>
+           LOGGER.error(e)
+           if (null != e.getMessage) {
+             sys.error("Exception occurred in query execution :: " + e.getMessage)
+           } else {
+             sys.error("Exception occurred in query execution.Please check logs.")
+           }
+       }
+
+       var havePair = false
+       var finished = false
+
+       override def hasNext: Boolean = {
+         if (!finished && !havePair) {
+           finished = (null == rowIterator) || (!rowIterator.hasNext())
+           havePair = !finished
+         }
+         !finished
+       }
+
+       override def next(): (K, V) = {
+         if (!hasNext) {
+           throw new java.util.NoSuchElementException("End of stream")
+         }
+         havePair = false
+         val row = rowIterator.next()
+         keyClass.getKey(row, null)
+       }
+
+       logInfo("********************** Total Time Taken to execute the query in Carbon Side: " +
+           (System.currentTimeMillis - queryStartTime)
+       )
+     }
+     iter
+   }
+
+   /**
+    * Get the preferred locations where to launch this task.
+    */
+  override def getPreferredLocations(partition: Partition): Seq[String] = {
+    val theSplit = partition.asInstanceOf[CarbonSparkPartition]
+    theSplit.locations.filter(_ != "localhost")
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a83dba34/integration/spark/src/test/scala/org/carbondata/spark/testsuite/joinquery/EquiJoinTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/joinquery/EquiJoinTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/joinquery/EquiJoinTestCase.scala
deleted file mode 100644
index 3943352..0000000
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/joinquery/EquiJoinTestCase.scala
+++ /dev/null
@@ -1,41 +0,0 @@
-package org.carbondata.spark.testsuite.joinquery
-
-import org.apache.spark.sql.common.util.CarbonHiveContext._
-import org.apache.spark.sql.common.util.QueryTest
-import org.scalatest.BeforeAndAfterAll
-import org.apache.spark.sql.execution.joins.BroadCastFilterPushJoin
-
-class EquiJoinTestCase extends QueryTest with BeforeAndAfterAll  {
-   override def beforeAll {
-    //loading to hive table
-    sql("create table employee_hive (empid string,empname string,mobilename string,mobilecolor string,salary int)row format delimited fields terminated by ','")
-    sql("create table mobile_hive (mobileid string,mobilename string, mobilecolor string, sales int)row format delimited fields terminated by ','");
-    sql("LOAD DATA LOCAL INPATH './src/test/resources/join/employee.csv' into table employee_hive")
-    sql("LOAD DATA LOCAL INPATH './src/test/resources/join/mobile.csv' into table mobile_hive")
-    //loading to carbon table
-    sql("create table employee (empid string,empname string,mobilename string,mobilecolor string,salary int) stored by 'org.apache.carbondata.format'")
-    sql("create table mobile (mobileid string,mobilename string, mobilecolor string, sales int) stored by 'org.apache.carbondata.format'");
-    sql("LOAD DATA LOCAL INPATH './src/test/resources/join/employee.csv' into table employee options('FILEHEADER'='empid,empname,mobilename,mobilecolor,salary')")
-    sql("LOAD DATA LOCAL INPATH './src/test/resources/join/mobile.csv' into table mobile options('FILEHEADER'='mobileid,mobilename,mobilecolor,sales')")
-   }
-   
-   test("test equijoin query") {
-     val df = sql("select employee.empname,mobile.mobilename from employee,mobile where employee.mobilename = mobile.mobilename")
-     var broadcastJoinExists = false
-     df.queryExecution.sparkPlan.collect {
-       case bcf: BroadCastFilterPushJoin =>
-         broadcastJoinExists = true
-     }
-     if (!broadcastJoinExists) {
-       assert(false)
-     }
-      checkAnswer(df,
-          sql("select employee_hive.empname,mobile_hive.mobilename from employee_hive,mobile_hive where employee_hive.mobilename = mobile_hive.mobilename"))
-  }
-  override def afterAll {
-    sql("drop table employee_hive")
-    sql("drop table mobile_hive")
-    sql("drop table employee")
-    sql("drop table mobile")
-  }
-}
\ No newline at end of file


[09/56] [abbrv] incubator-carbondata git commit: To ignore trmimming of leading and trailing whitespace the option for ignoring leading and trailing whitespace need to be set to false while create dataframe using univocity library (#681)

Posted by jb...@apache.org.
To ignore trmimming of leading and trailing whitespace the option for ignoring leading and trailing whitespace need to be set to false while create dataframe using univocity library (#681)



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

Branch: refs/heads/master
Commit: 075dd92cc7fd9694bbd67c1720f43a832ae2c53e
Parents: 25cd9e5
Author: manishgupta88 <to...@gmail.com>
Authored: Fri Jun 17 14:58:41 2016 +0800
Committer: david <Qi...@users.noreply.github.com>
Committed: Fri Jun 17 14:58:41 2016 +0800

----------------------------------------------------------------------
 .../scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/075dd92c/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
index 4526bb9..6298aaa 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -399,8 +399,8 @@ object GlobalDictionaryUtil extends Logging {
       })
       .option("parserLib", "univocity")
       .option("escape", carbonLoadModel.getEscapeChar)
-      .option("ignoreLeadingWhiteSpace", "true")
-      .option("ignoreTrailingWhiteSpace", "true")
+      .option("ignoreLeadingWhiteSpace", "false")
+      .option("ignoreTrailingWhiteSpace", "false")
       .load(carbonLoadModel.getFactFilePath)
     df
   }


[55/56] [abbrv] incubator-carbondata git commit: Refactor carbon-core module for code clean up (#720)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 70a515a..31d41b3 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -23,20 +23,13 @@ package org.carbondata.core.util;
 import java.io.Closeable;
 import java.io.DataInputStream;
 import java.io.File;
-import java.io.FileInputStream;
 import java.io.FileNotFoundException;
 import java.io.FileOutputStream;
 import java.io.IOException;
-import java.io.InputStream;
-import java.io.ObjectInputStream;
-import java.io.ObjectOutputStream;
-import java.io.OutputStream;
 import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.security.PrivilegedExceptionAction;
 import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.HashSet;
 import java.util.List;
@@ -55,9 +48,8 @@ import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.FileHolder;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
-import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreInfo;
 import org.carbondata.core.datastorage.store.columnar.UnBlockIndexer;
 import org.carbondata.core.datastorage.store.compression.MeasureMetaDataModel;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
@@ -65,12 +57,7 @@ import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.keygenerator.mdkey.NumberCompressor;
-import org.carbondata.core.metadata.BlockletInfo;
-import org.carbondata.core.metadata.BlockletInfoColumnar;
-import org.carbondata.core.metadata.SliceMetaData;
 import org.carbondata.core.metadata.ValueEncoderMeta;
-import org.carbondata.core.reader.CarbonFooterReader;
-import org.carbondata.core.vo.ColumnGroupModel;
 
 import org.apache.commons.lang.ArrayUtils;
 import org.apache.hadoop.conf.Configuration;
@@ -131,35 +118,6 @@ public final class CarbonUtil {
     }
   }
 
-  public static File[] getSortedFileList(File[] fileArray) {
-    Arrays.sort(fileArray, new Comparator<File>() {
-      public int compare(File o1, File o2) {
-        try {
-          return o1.getName().compareTo(o2.getName());
-        } catch (Exception e) {
-
-          LOGGER.error(e, "Error while getSortedFile");
-          return 0;
-        }
-      }
-    });
-    return fileArray;
-  }
-
-  public static CarbonFile[] getSortedFileList(CarbonFile[] fileArray) {
-    Arrays.sort(fileArray, new Comparator<CarbonFile>() {
-      public int compare(CarbonFile o1, CarbonFile o2) {
-        try {
-          return o1.getName().compareTo(o2.getName());
-        } catch (Exception e) {
-
-          return o1.getName().compareTo(o2.getName());
-        }
-      }
-    });
-    return fileArray;
-  }
-
   /**
    * @param baseStorePath
    * @return
@@ -425,35 +383,6 @@ public final class CarbonUtil {
    * @param path file path array
    * @throws Exception exception
    */
-  public static void deleteFoldersAndFiles(final String... path) throws CarbonUtilException {
-    if (path == null) {
-      return;
-    }
-    try {
-      UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Void>() {
-
-        @Override public Void run() throws Exception {
-          for (int i = 0; i < path.length; i++) {
-            if (null != path[i]) {
-              deleteRecursive(new File(path[i]));
-            }
-          }
-          return null;
-        }
-      });
-    } catch (IOException e) {
-      throw new CarbonUtilException("Error while deleting the folders and files");
-    } catch (InterruptedException e) {
-      throw new CarbonUtilException("Error while deleting the folders and files");
-    }
-  }
-
-  /**
-   * This method will be used to delete the folder and files
-   *
-   * @param path file path array
-   * @throws Exception exception
-   */
   public static void deleteFoldersAndFiles(final File... path) throws CarbonUtilException {
     try {
       UserGroupInformation.getLoginUser().doAs(new PrivilegedExceptionAction<Void>() {
@@ -571,10 +500,10 @@ public final class CarbonUtil {
           boolean isRenameSuccessfull = file.renameTo(newFilePath);
           if (!isRenameSuccessfull) {
             LOGGER.error("Problem renaming the cube :: " + fullPath);
-            c = new DeleteCube(file);
+            c = new DeleteFolderAndFiles(file);
             executorService.submit(c);
           } else {
-            c = new DeleteCube(FileFactory.getCarbonFile(newFilePath, fileType));
+            c = new DeleteFolderAndFiles(FileFactory.getCarbonFile(newFilePath, fileType));
             executorService.submit(c);
           }
         }
@@ -617,176 +546,6 @@ public final class CarbonUtil {
     }
   }
 
-  /**
-   * This method will be used to read leaf meta data format of meta data will be
-   * <entrycount><keylength><keyoffset><measure1length><measure1offset>
-   *
-   * @param file
-   * @param measureCount
-   * @param mdKeySize
-   * @return will return blocklet info which will have all the meta data
-   * related to data file
-   */
-  public static List<BlockletInfo> getBlockletInfo(File file, int measureCount, int mdKeySize) {
-    List<BlockletInfo> listOfBlockletInfo =
-        new ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    String filesLocation = file.getAbsolutePath();
-    long fileSize = file.length();
-    return getBlockletDetails(listOfBlockletInfo, filesLocation, measureCount, mdKeySize, fileSize);
-  }
-
-  /**
-   * This method will be used to read leaf meta data format of meta data will be
-   * <entrycount><keylength><keyoffset><measure1length><measure1offset>
-   *
-   * @param file
-   * @param measureCount
-   * @param mdKeySize
-   * @return will return blocklet info which will have all the meta data
-   * related to data file
-   */
-  public static List<BlockletInfo> getBlockletInfo(CarbonFile file, int measureCount,
-      int mdKeySize) {
-    List<BlockletInfo> listOfNodeInfo =
-        new ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    String filesLocation = file.getAbsolutePath();
-    long fileSize = file.getSize();
-    return getBlockletDetails(listOfNodeInfo, filesLocation, measureCount, mdKeySize, fileSize);
-  }
-
-  /**
-   * @param listOfNodeInfo
-   * @param filesLocation
-   * @param measureCount
-   * @param mdKeySize
-   * @param fileSize
-   * @return
-   */
-  private static List<BlockletInfo> getBlockletDetails(List<BlockletInfo> listOfNodeInfo,
-      String filesLocation, int measureCount, int mdKeySize, long fileSize) {
-    long offset = fileSize - CarbonCommonConstants.LONG_SIZE_IN_BYTE;
-    FileHolder fileHolder = FileFactory.getFileHolder(FileFactory.getFileType(filesLocation));
-    offset = fileHolder.readDouble(filesLocation, offset);
-    int totalMetaDataLength = (int) (fileSize - CarbonCommonConstants.LONG_SIZE_IN_BYTE - offset);
-    ByteBuffer buffer =
-        ByteBuffer.wrap(fileHolder.readByteArray(filesLocation, offset, totalMetaDataLength));
-    buffer.rewind();
-    while (buffer.hasRemaining()) {
-      int[] msrLength = new int[measureCount];
-      long[] msrOffset = new long[measureCount];
-      BlockletInfo info = new BlockletInfo();
-      byte[] startKey = new byte[mdKeySize];
-      byte[] endKey = new byte[mdKeySize];
-      info.setFileName(filesLocation);
-      info.setNumberOfKeys(buffer.getInt());
-      info.setKeyLength(buffer.getInt());
-      info.setKeyOffset(buffer.getLong());
-      buffer.get(startKey);
-      buffer.get(endKey);
-      info.setStartKey(startKey);
-      info.setEndKey(endKey);
-      for (int i = 0; i < measureCount; i++) {
-        msrLength[i] = buffer.getInt();
-        msrOffset[i] = buffer.getLong();
-      }
-      info.setMeasureLength(msrLength);
-      info.setMeasureOffset(msrOffset);
-      listOfNodeInfo.add(info);
-    }
-    fileHolder.finish();
-    return listOfNodeInfo;
-  }
-
-  /**
-   * This method will be used to read blocklet meta data format of meta data will
-   * be <entrycount><keylength><keyoffset><measure1length><measure1offset>
-   *
-   * @param file
-   * @return will return blocklet info which will have all the meta data
-   * related to leaf file
-   */
-  public static List<BlockletInfoColumnar> getBlockletInfoColumnar(CarbonFile file) {
-    List<BlockletInfoColumnar> listOfBlockletInfo =
-        new ArrayList<BlockletInfoColumnar>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    String filesLocation = file.getAbsolutePath();
-    long fileSize = file.getSize();
-    return getBlockletInfo(listOfBlockletInfo, filesLocation, fileSize);
-  }
-
-  /**
-   * @param listOfBlockletInfo
-   * @param filesLocation
-   * @param fileSize
-   * @return
-   */
-  private static List<BlockletInfoColumnar> getBlockletInfo(
-      List<BlockletInfoColumnar> listOfBlockletInfo, String filesLocation, long fileSize) {
-    long offset = fileSize - CarbonCommonConstants.LONG_SIZE_IN_BYTE;
-    FileHolder fileHolder = FileFactory.getFileHolder(FileFactory.getFileType(filesLocation));
-    offset = fileHolder.readDouble(filesLocation, offset);
-    CarbonFooterReader metaDataReader = new CarbonFooterReader(filesLocation, offset);
-    try {
-      listOfBlockletInfo = CarbonMetadataUtil.convertBlockletInfo(metaDataReader.readFooter());
-    } catch (IOException e) {
-      LOGGER.error("Problem while reading metadata :: " + filesLocation);
-    }
-    for (BlockletInfoColumnar infoColumnar : listOfBlockletInfo) {
-      infoColumnar.setFileName(filesLocation);
-    }
-    return listOfBlockletInfo;
-  }
-
-  /**
-   * This method will be used to read the slice metadata
-   *
-   * @param rsFiles
-   * @return slice meta data
-   * @throws CarbonUtilException
-   */
-  public static SliceMetaData readSliceMetadata(File rsFiles, int restructFolderNumber)
-      throws CarbonUtilException {
-    SliceMetaData readObject = null;
-    InputStream stream = null;
-    ObjectInputStream objectInputStream = null;
-    File file = null;
-    try {
-      file = new File(rsFiles + File.separator + getSliceMetaDataFileName(restructFolderNumber));
-      stream = new FileInputStream(
-          rsFiles + File.separator + getSliceMetaDataFileName(restructFolderNumber));
-      objectInputStream = new ObjectInputStream(stream);
-      readObject = (SliceMetaData) objectInputStream.readObject();
-    } catch (ClassNotFoundException e) {
-      throw new CarbonUtilException(
-          "Problem while reading the slicemeta data file " + file.getAbsolutePath(), e);
-    }
-    //
-    catch (IOException e) {
-      throw new CarbonUtilException("Problem while reading the slicemeta data file ", e);
-    } finally {
-      closeStreams(objectInputStream, stream);
-    }
-    return readObject;
-  }
-
-  public static void writeSliceMetaDataFile(String path, SliceMetaData sliceMetaData,
-      int nextRestructFolder) {
-    OutputStream stream = null;
-    ObjectOutputStream objectOutputStream = null;
-    try {
-      LOGGER.info("Slice Metadata file Path: " + path + '/' + CarbonUtil
-          .getSliceMetaDataFileName(nextRestructFolder));
-      stream = FileFactory
-          .getDataOutputStream(path + File.separator + getSliceMetaDataFileName(nextRestructFolder),
-              FileFactory.getFileType(path));
-      objectOutputStream = new ObjectOutputStream(stream);
-      objectOutputStream.writeObject(sliceMetaData);
-    } catch (IOException e) {
-      LOGGER.error(e.getMessage());
-    } finally {
-      closeStreams(objectOutputStream, stream);
-    }
-  }
-
   public static void deleteFiles(File[] intermediateFiles) throws CarbonUtilException {
     for (int i = 0; i < intermediateFiles.length; i++) {
       if (!intermediateFiles[i].delete()) {
@@ -795,27 +554,6 @@ public final class CarbonUtil {
     }
   }
 
-  public static ColumnarKeyStoreInfo getColumnarKeyStoreInfo(BlockletInfoColumnar blockletInfo,
-      int[] eachBlockSize, ColumnGroupModel colGrpModel) {
-    ColumnarKeyStoreInfo columnarKeyStoreInfo = new ColumnarKeyStoreInfo();
-    columnarKeyStoreInfo.setFilePath(blockletInfo.getFileName());
-    columnarKeyStoreInfo.setIsSorted(blockletInfo.getIsSortedKeyColumn());
-    columnarKeyStoreInfo.setKeyBlockIndexLength(blockletInfo.getKeyBlockIndexLength());
-    columnarKeyStoreInfo.setKeyBlockIndexOffsets(blockletInfo.getKeyBlockIndexOffSets());
-    columnarKeyStoreInfo.setKeyBlockLengths(blockletInfo.getKeyLengths());
-    columnarKeyStoreInfo.setKeyBlockOffsets(blockletInfo.getKeyOffSets());
-    columnarKeyStoreInfo.setNumberOfKeys(blockletInfo.getNumberOfKeys());
-    columnarKeyStoreInfo.setSizeOfEachBlock(eachBlockSize);
-    columnarKeyStoreInfo.setNumberCompressor(new NumberCompressor(Integer.parseInt(
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.BLOCKLET_SIZE,
-            CarbonCommonConstants.BLOCKLET_SIZE_DEFAULT_VAL))));
-    columnarKeyStoreInfo.setAggKeyBlock(blockletInfo.getAggKeyBlock());
-    columnarKeyStoreInfo.setDataIndexMapLength(blockletInfo.getDataIndexMapLength());
-    columnarKeyStoreInfo.setDataIndexMapOffsets(blockletInfo.getDataIndexMapOffsets());
-    columnarKeyStoreInfo.setHybridStoreModel(colGrpModel);
-    return columnarKeyStoreInfo;
-  }
-
   public static byte[] getKeyArray(ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolder,
       int totalKeySize, int eachKeySize) {
     byte[] completeKeyArray = new byte[totalKeySize];
@@ -918,103 +656,6 @@ public final class CarbonUtil {
     return integers;
   }
 
-  public static String[] getSlices(String storeLocation,
-      FileFactory.FileType fileType) {
-    try {
-      if (!FileFactory.isFileExist(storeLocation, fileType)) {
-        return new String[0];
-      }
-    } catch (IOException e) {
-      LOGGER.error("Error occurred :: " + e.getMessage());
-    }
-    CarbonFile file = FileFactory.getCarbonFile(storeLocation, fileType);
-    CarbonFile[] listFiles = listFiles(file);
-    if (null == listFiles || listFiles.length < 0) {
-      return new String[0];
-    }
-    Arrays.sort(listFiles, new CarbonFileFolderComparator());
-    String[] slices = new String[listFiles.length];
-    for (int i = 0; i < listFiles.length; i++) {
-      slices[i] = listFiles[i].getAbsolutePath();
-    }
-    return slices;
-  }
-
-  /**
-   * @param file
-   * @return
-   */
-  public static CarbonFile[] listFiles(CarbonFile file) {
-    CarbonFile[] listFiles = file.listFiles(new CarbonFileFilter() {
-      @Override public boolean accept(CarbonFile pathname) {
-        return pathname.getName().startsWith(CarbonCommonConstants.LOAD_FOLDER) && !pathname
-            .getName().endsWith(CarbonCommonConstants.FILE_INPROGRESS_STATUS);
-      }
-    });
-    return listFiles;
-  }
-
-  public static List<CarbonSliceAndFiles> getSliceAndFilesList(String tableName,
-      CarbonFile[] listFiles, FileFactory.FileType fileType) {
-
-    List<CarbonSliceAndFiles> sliceFactFilesList =
-        new ArrayList<CarbonSliceAndFiles>(listFiles.length);
-    if (listFiles.length == 0) {
-      return sliceFactFilesList;
-    }
-
-    CarbonSliceAndFiles sliceAndFiles = null;
-    CarbonFile[] sortedPathForFiles = null;
-    for (int i = 0; i < listFiles.length; i++) {
-      sliceAndFiles = new CarbonSliceAndFiles();
-      sliceAndFiles.setPath(listFiles[i].getAbsolutePath());
-      sortedPathForFiles = getAllFactFiles(sliceAndFiles.getPath(), tableName, fileType);
-      if (null != sortedPathForFiles && sortedPathForFiles.length > 0) {
-        Arrays.sort(sortedPathForFiles,
-            new CarbonFileComparator("\\" + CarbonCommonConstants.FACT_FILE_EXT));
-        sliceAndFiles.setSliceFactFilesList(sortedPathForFiles);
-        sliceFactFilesList.add(sliceAndFiles);
-      }
-    }
-    return sliceFactFilesList;
-  }
-
-  /**
-   * Below method will be used to get the fact file present in slice
-   *
-   * @param sliceLocation slice location
-   * @return fact files array
-   */
-  public static CarbonFile[] getAllFactFiles(String sliceLocation, final String tableName,
-      FileFactory.FileType fileType) {
-    CarbonFile file = FileFactory.getCarbonFile(sliceLocation, fileType);
-    CarbonFile[] files = null;
-    CarbonFile[] updatedFactFiles = null;
-    if (file.isDirectory()) {
-      updatedFactFiles = file.listFiles(new CarbonFileFilter() {
-
-        @Override public boolean accept(CarbonFile pathname) {
-          return ((!pathname.isDirectory()) && (pathname.getName().startsWith(tableName))
-              && pathname.getName().endsWith(CarbonCommonConstants.FACT_UPDATE_EXTENSION));
-        }
-      });
-
-      if (updatedFactFiles.length != 0) {
-        return updatedFactFiles;
-
-      }
-
-      files = file.listFiles(new CarbonFileFilter() {
-        public boolean accept(CarbonFile pathname) {
-          return ((!pathname.isDirectory()) && (pathname.getName().startsWith(tableName))
-              && pathname.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT));
-
-        }
-      });
-    }
-    return files;
-  }
-
   /**
    * Read level metadata file and return cardinality
    *
@@ -1049,61 +690,6 @@ public final class CarbonUtil {
     return cardinality;
   }
 
-  public static String getNewAggregateTableName(List<String> tableList, String factTableName) {
-    int count = 1;
-    List<String> newTableList = new ArrayList<String>(10);
-    newTableList.addAll(tableList);
-    if (newTableList.contains(factTableName)) {
-      newTableList.remove(factTableName);
-    }
-    if (!newTableList.isEmpty()) {
-      Collections.sort(newTableList, new AggTableComparator());
-      String highestCountAggTableName = newTableList.get(0);
-      count = Integer.parseInt(
-          highestCountAggTableName.substring(highestCountAggTableName.lastIndexOf("_") + 1))
-          + count;
-    }
-    return CarbonCommonConstants.AGGREGATE_TABLE_START_TAG + CarbonCommonConstants.UNDERSCORE
-        + factTableName + CarbonCommonConstants.UNDERSCORE + count;
-  }
-
-  public static String getRSPath(String schemaName, String cubeName, String tableName,
-      String hdfsLocation, int currentRestructNumber) {
-    if (null == hdfsLocation) {
-      hdfsLocation =
-          CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION_HDFS);
-    }
-
-    String hdfsStoreLocation = hdfsLocation;
-    hdfsStoreLocation = hdfsStoreLocation + File.separator + schemaName + File.separator + cubeName;
-
-    int rsCounter = currentRestructNumber/*CarbonUtil.checkAndReturnNextRestructFolderNumber(
-                hdfsStoreLocation, "RS_")*/;
-    if (rsCounter == -1) {
-      rsCounter = 0;
-    }
-    String hdfsLoadedTable =
-        hdfsStoreLocation + File.separator + CarbonCommonConstants.RESTRUCTRE_FOLDER + rsCounter
-            + "/" + tableName;
-    return hdfsLoadedTable;
-  }
-
-  public static boolean createRSMetaFile(String metaDataPath, String newRSFileName) {
-    String fullFileName = metaDataPath + File.separator + newRSFileName;
-    FileFactory.FileType fileType =
-        FileFactory.getFileType(metaDataPath + File.separator + newRSFileName);
-    try {
-      return FileFactory.createNewFile(fullFileName, fileType);
-    } catch (IOException e) {
-      LOGGER.error("Error while writing RS meta file : " + fullFileName + e.getMessage());
-      return false;
-    }
-  }
-
-  public static String getSliceMetaDataFileName(int restructFolderNumber) {
-    return CarbonCommonConstants.SLICE_METADATA_FILENAME + "." + restructFolderNumber;
-  }
-
   public static void writeLevelCardinalityFile(String loadFolderLoc, String tableName,
       int[] dimCardinality) throws KettleException {
     String levelCardinalityFilePath = loadFolderLoc + File.separator +
@@ -1140,32 +726,6 @@ public final class CarbonUtil {
     }
   }
 
-  public static SliceMetaData readSliceMetaDataFile(String path) {
-    SliceMetaData readObject = null;
-    InputStream stream = null;
-    ObjectInputStream objectInputStream = null;
-    //
-    try {
-      stream = FileFactory.getDataInputStream(path, FileFactory.getFileType(path));
-      objectInputStream = new ObjectInputStream(stream);
-      readObject = (SliceMetaData) objectInputStream.readObject();
-    } catch (ClassNotFoundException e) {
-      LOGGER.error(e);
-    } catch (FileNotFoundException e) {
-      LOGGER.error("@@@@@ SliceMetaData File is missing @@@@@ :" + path);
-    } catch (IOException e) {
-      LOGGER.error("@@@@@ Error while reading SliceMetaData File @@@@@ :" + path);
-    } finally {
-      closeStreams(objectInputStream, stream);
-    }
-    return readObject;
-  }
-
-  public static SliceMetaData readSliceMetaDataFile(String folderPath, int currentRestructNumber) {
-    String path = folderPath + '/' + getSliceMetaDataFileName(currentRestructNumber);
-    return readSliceMetaDataFile(path);
-  }
-
   /**
    * From beeline if a delimeter is passed as \001, in code we get it as
    * escaped string as \\001. So this method will unescape the slash again and
@@ -1248,26 +808,6 @@ public final class CarbonUtil {
   }
 
   /**
-   * This method will read the retry time interval for loading level files in
-   * memory
-   *
-   * @return
-   */
-  public static long getRetryIntervalForLoadingLevelFile() {
-    long retryInterval = 0;
-    try {
-      retryInterval = Long.parseLong(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_LOAD_LEVEL_RETRY_INTERVAL,
-              CarbonCommonConstants.CARBON_LOAD_LEVEL_RETRY_INTERVAL_DEFAULT));
-    } catch (NumberFormatException e) {
-      retryInterval = Long.parseLong(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_LOAD_LEVEL_RETRY_INTERVAL_DEFAULT));
-    }
-    retryInterval = retryInterval * 1000;
-    return retryInterval;
-  }
-
-  /**
    * Below method will be used to get the aggregator type
    * CarbonCommonConstants.SUM_COUNT_VALUE_MEASURE will return when value is double measure
    * CarbonCommonConstants.BYTE_VALUE_MEASURE will be returned when value is byte array
@@ -1500,12 +1040,11 @@ public final class CarbonUtil {
   /**
    * Thread to delete the cubes
    *
-   * @author m00258959
    */
-  private static final class DeleteCube implements Callable<Void> {
+  private static final class DeleteFolderAndFiles implements Callable<Void> {
     private CarbonFile file;
 
-    private DeleteCube(CarbonFile file) {
+    private DeleteFolderAndFiles(CarbonFile file) {
       this.file = file;
     }
 
@@ -1516,34 +1055,6 @@ public final class CarbonUtil {
 
   }
 
-  private static class CarbonFileComparator implements Comparator<CarbonFile> {
-    /**
-     * File extension
-     */
-    private String fileExt;
-
-    public CarbonFileComparator(String fileExt) {
-      this.fileExt = fileExt;
-    }
-
-    @Override public int compare(CarbonFile file1, CarbonFile file2) {
-      String firstFileName = file1.getName().split(fileExt)[0];
-      String secondFileName = file2.getName().split(fileExt)[0];
-      int lastIndexOfO1 = firstFileName.lastIndexOf('_');
-      int lastIndexOfO2 = secondFileName.lastIndexOf('_');
-      int f1 = 0;
-      int f2 = 0;
-
-      try {
-        f1 = Integer.parseInt(firstFileName.substring(lastIndexOfO1 + 1));
-        f2 = Integer.parseInt(secondFileName.substring(lastIndexOfO2 + 1));
-      } catch (NumberFormatException nfe) {
-        return -1;
-      }
-      return (f1 < f2) ? -1 : (f1 == f2 ? 0 : 1);
-    }
-  }
-
   /**
    * class to sort aggregate folder list in descending order
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java b/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
deleted file mode 100644
index b3b9d37..0000000
--- a/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
+++ /dev/null
@@ -1,137 +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.carbondata.core.vo;
-
-public class ColumnGroupModel {
-
-  /**
-   * cardinality all dimension
-   */
-  private int[] colGroupCardinality;
-
-  /**
-   * number of columns in columnar block
-   */
-  private int[] columnSplit;
-
-  /**
-   * total number of columns
-   */
-  private int noOfColumnsStore;
-
-  /**
-   * whether given index is columnar or not
-   * true: columnar
-   * false: row block
-   */
-  private boolean[] columnarStore;
-
-  /**
-   * column groups
-   * e.g
-   * {{0,1,2},3,4,{5,6}}
-   */
-  private int[][] columnGroups;
-
-  /**
-   * @return cardinality of column groups
-   */
-  public int[] getColumnGroupCardinality() {
-    return colGroupCardinality;
-  }
-
-  /**
-   * set columngroup cardinality
-   *
-   * @param columnGroupCardinality
-   */
-  public void setColumnGroupCardinality(int[] columnGroupCardinality) {
-    this.colGroupCardinality = columnGroupCardinality;
-  }
-
-  /**
-   * return columnSplit
-   *
-   * @return
-   */
-  public int[] getColumnSplit() {
-    return columnSplit;
-  }
-
-  /**
-   * set columnSplit
-   *
-   * @param split
-   */
-  public void setColumnSplit(int[] split) {
-    this.columnSplit = split;
-  }
-
-  /**
-   * @return no of columnar block
-   */
-  public int getNoOfColumnStore() {
-    return this.noOfColumnsStore;
-  }
-
-  /**
-   * set no of columnar block
-   *
-   * @param noOfColumnsStore
-   */
-  public void setNoOfColumnStore(int noOfColumnsStore) {
-    this.noOfColumnsStore = noOfColumnsStore;
-  }
-
-  /**
-   * it's an identifier for row block or single column block
-   *
-   * @param columnarStore
-   */
-  public void setColumnarStore(boolean[] columnarStore) {
-    this.columnarStore = columnarStore;
-  }
-
-  /**
-   * set column groups
-   *
-   * @param columnGroups
-   */
-  public void setColumnGroup(int[][] columnGroups) {
-    this.columnGroups = columnGroups;
-  }
-
-  /**
-   * check if given column group is columnar
-   *
-   * @param colGroup
-   * @return true if given block is columnar
-   */
-  public boolean isColumnar(int colGroup) {
-    return columnarStore[colGroup];
-  }
-
-  /**
-   * @return columngroups
-   */
-  public int[][] getColumnGroup() {
-    return this.columnGroups;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java b/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java
deleted file mode 100644
index 0f31c45..0000000
--- a/core/src/main/java/org/carbondata/core/writer/CarbonDataWriter.java
+++ /dev/null
@@ -1,482 +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.carbondata.core.writer;
-
-import java.io.DataOutputStream;
-import java.io.File;
-import java.io.FileFilter;
-import java.io.FileNotFoundException;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.text.MessageFormat;
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.file.manager.composite.FileData;
-import org.carbondata.core.file.manager.composite.IFileManagerComposite;
-import org.carbondata.core.metadata.BlockletInfo;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.core.writer.exception.CarbonDataWriterException;
-
-public class CarbonDataWriter {
-  /**
-   * Attribute for Carbon LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(CarbonDataWriter.class.getName());
-  /**
-   * table name
-   */
-  private String tableName;
-  /**
-   * data file size;
-   */
-  private long fileSizeInBytes;
-  /**
-   * measure count
-   */
-  private int measureCount;
-  /**
-   * this will be used for holding blocklet metadata
-   */
-  private List<BlockletInfo> blockletInfoList;
-  /**
-   * current size of file
-   */
-  private long currentFileSize;
-  /**
-   * leaf metadata size
-   */
-  private int leafMetaDataSize;
-  /**
-   * file count will be used to give sequence number to the data file
-   */
-  private int fileCount;
-  /**
-   * filename format
-   */
-  private String fileNameFormat;
-  /**
-   * file name
-   */
-  private String fileName;
-  /**
-   * File manager
-   */
-  private IFileManagerComposite fileManager;
-  /**
-   * Store Location
-   */
-  private String storeLocation;
-  /**
-   * fileExtension
-   */
-  private String fileExtension;
-  /**
-   * isNewFileCreationRequired
-   */
-  private boolean isNewFileCreationRequired;
-  /**
-   * isInProgressExtrequired
-   */
-  private boolean isInProgressExtrequired;
-  /**
-   * fileDataOutStream
-   */
-  private DataOutputStream fileDataOutStream;
-  /**
-   * metadataOffset for maintaining the offset of pagination file.
-   */
-  private int metadataOffset;
-
-  /**
-   * CarbonDataWriter constructor to initialize all the instance variables
-   * required for wrting the data i to the file
-   *
-   * @param storeLocation current store location
-   * @param measureCount  total number of measures
-   * @param mdKeyLength   mdkey length
-   * @param tableName     table name
-   */
-  public CarbonDataWriter(String storeLocation, int measureCount, int mdKeyLength, String tableName,
-      String fileExtension, boolean isNewFileCreationRequired, boolean isInProgressExtrequired) {
-    // measure count
-    this.measureCount = measureCount;
-    // table name
-    this.tableName = tableName;
-
-    this.storeLocation = storeLocation;
-    this.fileExtension = fileExtension;
-    // create the carbon file format
-    this.fileNameFormat =
-        storeLocation + File.separator + this.tableName + '_' + "{0}" + this.fileExtension;
-
-    this.leafMetaDataSize = CarbonCommonConstants.INT_SIZE_IN_BYTE * (2 + measureCount)
-        + CarbonCommonConstants.LONG_SIZE_IN_BYTE * (measureCount + 1) + (2 * mdKeyLength);
-    this.blockletInfoList = new ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-    // get max file size;
-    this.fileSizeInBytes = Long.parseLong(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.MAX_FILE_SIZE,
-            CarbonCommonConstants.MAX_FILE_SIZE_DEFAULT_VAL))
-        * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR
-        * CarbonCommonConstants.BYTE_TO_KB_CONVERSION_FACTOR * 1L;
-    this.isNewFileCreationRequired = isNewFileCreationRequired;
-    this.isInProgressExtrequired = isInProgressExtrequired;
-  }
-
-  /**
-   * This method will be used to initialize the channel
-   *
-   * @throws CarbonDataWriterException
-   */
-  public void initChannel() throws CarbonDataWriterException {
-    // update the filename with new new sequence
-    // increment the file sequence counter
-    initFileCount();
-    if (this.isInProgressExtrequired) {
-      this.fileName = MessageFormat.format(this.fileNameFormat, this.fileCount)
-          + CarbonCommonConstants.FILE_INPROGRESS_STATUS;
-      FileData fileData = new FileData(this.fileName, this.storeLocation);
-      fileManager.add(fileData);
-    } else {
-      this.fileName = MessageFormat.format(this.fileNameFormat, this.fileCount);
-    }
-    this.fileCount++;
-    try {
-      // open stream for new data file
-      this.fileDataOutStream = FileFactory
-          .getDataOutputStream(this.fileName, FileFactory.getFileType(this.fileName), (short) 1);
-    } catch (FileNotFoundException fileNotFoundException) {
-      throw new CarbonDataWriterException("Problem while getting the writer for Leaf File",
-          fileNotFoundException);
-    } catch (IOException e) {
-      throw new CarbonDataWriterException("Problem while getting the writer for Leaf File", e);
-    }
-  }
-
-  /**
-   * Method will be used to close the open stream
-   */
-  public CarbonFile closeChannle() {
-    CarbonUtil.closeStreams(this.fileDataOutStream);
-
-    CarbonFile carbonFile = FileFactory.getCarbonFile(fileName, FileFactory.getFileType(fileName));
-
-    if (!carbonFile.renameTo(fileName.substring(0, this.fileName.lastIndexOf('.')))) {
-      LOGGER.info("file renaming failed from _0.querymerged to _0");
-    }
-
-    return carbonFile;
-  }
-
-  private int initFileCount() {
-    int fileCnt = 0;
-    File[] dataFiles = new File(storeLocation).listFiles(new FileFilter() {
-
-      @Override public boolean accept(File file) {
-        if (!file.isDirectory() && file.getName().startsWith(tableName) && file.getName()
-            .contains(fileExtension)) {
-          return true;
-        }
-        return false;
-      }
-    });
-    if (dataFiles != null && dataFiles.length > 0) {
-      Arrays.sort(dataFiles);
-      String fileName = dataFiles[dataFiles.length - 1].getName();
-      try {
-        fileCnt =
-            Integer.parseInt(fileName.substring(fileName.lastIndexOf('_') + 1).split("\\.")[0]);
-      } catch (NumberFormatException ex) {
-        fileCnt = 0;
-      }
-      fileCnt++;
-    }
-    return fileCnt;
-  }
-
-  /**
-   * This method will be used to update the file channel with new file; new
-   * file will be created once existing file reached the file size limit This
-   * method will first check whether existing file size is exceeded the file
-   * size limit if yes then write the blocklet metadata to file then set the
-   * current file size to 0 close the existing file channel get the new file
-   * name and get the channel for new file
-   *
-   * @throws CarbonDataWriterException if any problem
-   */
-  private void updateBlockletFileChannel() throws CarbonDataWriterException {
-    // get the current file size exceeding the file size threshold
-    if (currentFileSize >= fileSizeInBytes) {
-      // write meta data to end of the existing file
-      writeBlockletMetaDataToFile();
-      // set the current file size;
-      this.currentFileSize = 0;
-      // close the current open file channel
-      CarbonUtil.closeStreams(fileDataOutStream);
-      // initialize the new channel
-      initChannel();
-    }
-  }
-
-  /**
-   * This method will be used to write leaf data to file
-   * file format
-   * <key><measure1><measure2>....
-   *
-   * @param keyArray   key array
-   * @param dataArray  measure array
-   * @param entryCount number of entries
-   * @param startKey   start key of leaf
-   * @param endKey     end key of leaf
-   * @throws CarbonDataWriterException
-   * @throws CarbonDataWriterException throws new CarbonDataWriterException if any problem
-   */
-  public void writeDataToFile(byte[] keyArray, byte[][] dataArray, int entryCount, byte[] startKey,
-      byte[] endKey) throws CarbonDataWriterException {
-    if (this.isNewFileCreationRequired) {
-      updateBlockletFileChannel();
-    }
-    // total measure length;
-    int totalMsrArraySize = 0;
-    // current measure length;
-    int currentMsrLenght = 0;
-    int[] msrLength = new int[this.measureCount];
-
-    // calculate the total size required for all the measure and get the
-    // each measure size
-    for (int i = 0; i < dataArray.length; i++) {
-      currentMsrLenght = dataArray[i].length;
-      totalMsrArraySize += currentMsrLenght;
-      msrLength[i] = currentMsrLenght;
-    }
-    byte[] writableDataArray = new byte[totalMsrArraySize];
-
-    // start position will be used for adding the measure in
-    // writableDataArray after adding measure increment the start position
-    // by added measure length which will be used for next measure start
-    // position
-    int startPosition = 0;
-    for (int i = 0; i < dataArray.length; i++) {
-      System.arraycopy(dataArray[i], 0, writableDataArray, startPosition, dataArray[i].length);
-      startPosition += msrLength[i];
-    }
-    writeDataToFile(keyArray, writableDataArray, msrLength, entryCount, startKey, endKey);
-  }
-
-  /**
-   * This method will be used to write leaf data to file
-   * file format
-   * <key><measure1><measure2>....
-   *
-   * @param keyArray   key array
-   * @param dataArray  measure array
-   * @param entryCount number of entries
-   * @param startKey   start key of leaf
-   * @param endKey     end key of leaf
-   * @throws CarbonDataWriterException
-   * @throws CarbonDataWriterException throws new CarbonDataWriterException if any problem
-   */
-  public void writeDataToFile(byte[] keyArray, byte[] dataArray, int[] msrLength, int entryCount,
-      byte[] startKey, byte[] endKey) throws CarbonDataWriterException {
-    int keySize = keyArray.length;
-    // write data to leaf file and get its offset
-    long offset = writeDataToFile(keyArray, dataArray);
-
-    // get the blocklet info for currently added blocklet
-    BlockletInfo blockletInfo =
-        getBlockletInfo(keySize, msrLength, offset, entryCount, startKey, endKey);
-    // add blocklet info to list
-    this.blockletInfoList.add(blockletInfo);
-    // calculate the current size of the file
-    this.currentFileSize +=
-        keySize + dataArray.length + (blockletInfoList.size() * this.leafMetaDataSize)
-            + CarbonCommonConstants.LONG_SIZE_IN_BYTE;
-  }
-
-  /**
-   * This method will be used to get the blocklet metadata
-   *
-   * @param keySize    key size
-   * @param msrLength  measure length array
-   * @param offset     current offset
-   * @param entryCount total number of rows in leaf
-   * @param startKey   start key of leaf
-   * @param endKey     end key of leaf
-   * @return BlockletInfo - leaf metadata
-   */
-  private BlockletInfo getBlockletInfo(int keySize, int[] msrLength, long offset, int entryCount,
-      byte[] startKey, byte[] endKey) {
-    // create the info object for leaf entry
-    BlockletInfo info = new BlockletInfo();
-    // add total entry count
-    info.setNumberOfKeys(entryCount);
-
-    // add the key array length
-    info.setKeyLength(keySize);
-
-    // add key offset
-    info.setKeyOffset(offset);
-
-    // increment the current offset by adding key length to get the measure
-    // offset position
-    // format of metadata will be
-    // <entrycount>,<keylenght>,<keyoffset>,<msr1lenght><msr1offset><msr2length><msr2offset>
-    offset += keySize;
-
-    // add measure length
-    info.setMeasureLength(msrLength);
-
-    long[] msrOffset = new long[this.measureCount];
-
-    for (int i = 0; i < this.measureCount; i++) {
-      msrOffset[i] = offset;
-      // now increment the offset by adding measure length to get the next
-      // measure offset;
-      offset += msrLength[i];
-    }
-    // add measure offset
-    info.setMeasureOffset(msrOffset);
-    // set startkey
-    info.setStartKey(startKey);
-    // set end key
-    info.setEndKey(endKey);
-    // return leaf metadata
-    return info;
-  }
-
-  /**
-   * This method is responsible for writing blocklet to the data file
-   *
-   * @param keyArray     mdkey array
-   * @param measureArray measure array
-   * @return file offset offset is the current position of the file
-   * @throws CarbonDataWriterException if will throw CarbonDataWriterException when any thing
-   *                                   goes wrong while while writing the leaf file
-   */
-  private long writeDataToFile(byte[] keyArray, byte[] measureArray)
-      throws CarbonDataWriterException {
-    long offset = metadataOffset;
-    try {
-      metadataOffset += keyArray.length + measureArray.length;
-      this.fileDataOutStream.write(keyArray);
-      this.fileDataOutStream.write(measureArray);
-    } catch (IOException exception) {
-      throw new CarbonDataWriterException("Problem in writing carbon file: ", exception);
-    }
-    // return the offset, this offset will be used while reading the file in
-    // engine side to get from which position to start reading the file
-    return offset;
-  }
-
-  /**
-   * This method will write metadata at the end of file file format
-   * <KeyArray><measure1><measure2> <KeyArray><measure1><measure2>
-   * <KeyArray><measure1><measure2> <KeyArray><measure1><measure2>
-   * <entrycount>
-   * <keylength><keyoffset><measure1length><measure1offset><measure2length
-   * ><measure2offset>
-   *
-   * @throws CarbonDataWriterException throw CarbonDataWriterException when problem in
-   *                                   writing the meta data to file
-   */
-  public void writeBlockletMetaDataToFile() throws CarbonDataWriterException {
-    ByteBuffer buffer = null;
-    int[] msrLength = null;
-    long[] msroffset = null;
-    try {
-      // get the current position of the file, this will be used for
-      // reading the file meta data, meta data start position in file will
-      // be this position
-      for (BlockletInfo info : this.blockletInfoList) {
-        // get the measure length array
-        msrLength = info.getMeasureLength();
-        // get the measure offset array
-        msroffset = info.getMeasureOffset();
-        // allocate total size for buffer
-        buffer = ByteBuffer.allocate(this.leafMetaDataSize);
-        // add entry count
-        buffer.putInt(info.getNumberOfKeys());
-        // add key length
-        buffer.putInt(info.getKeyLength());
-        // add key offset
-        buffer.putLong(info.getKeyOffset());
-        // set the start key
-        buffer.put(info.getStartKey());
-        // set the end key
-        buffer.put(info.getEndKey());
-        // add each measure length and its offset
-        for (int i = 0; i < this.measureCount; i++) {
-          buffer.putInt(msrLength[i]);
-          buffer.putLong(msroffset[i]);
-        }
-        // flip the buffer
-        buffer.flip();
-        // write metadat to file
-        this.fileDataOutStream.write(buffer.array());
-      }
-      // create new for adding the offset of meta data
-      // write offset to file
-      this.fileDataOutStream.writeLong(metadataOffset);
-    } catch (IOException exception) {
-      throw new CarbonDataWriterException("Problem while writing the carbon file: ", exception);
-    }
-    // create new blocklet info list for new file
-    this.blockletInfoList = new ArrayList<BlockletInfo>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
-  }
-
-  /**
-   * This method will be used to get the leaf meta list size
-   *
-   * @return list size
-   */
-  public int getMetaListSize() {
-    return blockletInfoList.size();
-  }
-
-  public void setFileManager(IFileManagerComposite fileManager) {
-    this.fileManager = fileManager;
-  }
-
-  /**
-   * getFileCount
-   *
-   * @return int
-   */
-  public int getFileCount() {
-    return fileCount;
-  }
-
-  /**
-   * setFileCount
-   *
-   * @param fileCount void
-   */
-  public void setFileCount(int fileCount) {
-    this.fileCount = fileCount;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
index 6e04714..eea9988 100644
--- a/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
+++ b/core/src/main/java/org/carbondata/scan/executor/QueryExecutor.java
@@ -18,7 +18,7 @@
  */
 package org.carbondata.scan.executor;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.model.QueryModel;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
index 8232567..edb7675 100644
--- a/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/DetailQueryExecutor.java
@@ -20,7 +20,7 @@ package org.carbondata.scan.executor.impl;
 
 import java.util.List;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
index 1ce0a36..8659e82 100644
--- a/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
+++ b/core/src/main/java/org/carbondata/scan/executor/impl/DetailRawRecordQueryExecutor.java
@@ -2,7 +2,7 @@ package org.carbondata.scan.executor.impl;
 
 import java.util.List;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.executor.exception.QueryExecutionException;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java b/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
index e91e6de..def2725 100644
--- a/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
@@ -90,10 +90,6 @@ public class RestructureFilterResolverImpl implements FilterResolverIntf {
           } else {
             dimColumnResolvedFilterInfo
                 .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
             dimColumnResolvedFilterInfo.setFilterValues(
                 FilterUtil.getFilterListForRS(right, columnExpression, defaultValue, surrogate));
           }
@@ -115,10 +111,6 @@ public class RestructureFilterResolverImpl implements FilterResolverIntf {
           } else {
             dimColumnResolvedFilterInfo
                 .setColumnIndex(columnExpression.getCarbonColumn().getOrdinal());
-            // dimColumnResolvedFilterInfo
-            // .setNeedCompressedData(info.getSlices().get(info.getCurrentSliceIndex())
-            // .getDataCache(info.getFactTableName()).getAggKeyBlock()[columnExpression.getDim()
-            // .getOrdinal()]);
             dimColumnResolvedFilterInfo.setFilterValues(
                 FilterUtil.getFilterListForRS(left, columnExpression, defaultValue, surrogate));
           }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
index e3c7cd5..e385bec 100644
--- a/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
+++ b/core/src/main/java/org/carbondata/scan/processor/AbstractDataBlockIterator.java
@@ -18,11 +18,11 @@
  */
 package org.carbondata.scan.processor;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.DataRefNode;
 import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.scan.collector.ScannedResultCollector;
 import org.carbondata.scan.collector.impl.ListBasedResultCollector;
 import org.carbondata.scan.executor.exception.QueryExecutionException;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
index 0b36ab5..7cecf64 100644
--- a/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
+++ b/core/src/main/java/org/carbondata/scan/processor/BlockletIterator.java
@@ -19,8 +19,8 @@
 
 package org.carbondata.scan.processor;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.iterator.CarbonIterator;
 
 /**
  * Below class will be used to iterate over data block

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/BatchResult.java b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
index dc14060..c5e1182 100644
--- a/core/src/main/java/org/carbondata/scan/result/BatchResult.java
+++ b/core/src/main/java/org/carbondata/scan/result/BatchResult.java
@@ -21,7 +21,7 @@ package org.carbondata.scan.result;
 
 import java.util.NoSuchElementException;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 
 /**
  * Below class holds the query result

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
index 2356a9f..1b49bcc 100644
--- a/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -20,6 +20,7 @@ package org.carbondata.scan.result.iterator;
 
 import java.util.List;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.DataRefNode;
@@ -28,7 +29,6 @@ import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.scan.executor.infos.BlockExecutionInfo;
 import org.carbondata.scan.model.QueryModel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
index 5cc4f1e..63a7a96 100644
--- a/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/ChunkRowIterator.java
@@ -19,7 +19,7 @@
 
 package org.carbondata.scan.result.iterator;
 
-import org.carbondata.core.iterator.CarbonIterator;
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.scan.result.BatchResult;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
index 8c028b2..62849ee 100644
--- a/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
+++ b/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
@@ -18,10 +18,10 @@
  */
 package org.carbondata.scan.result.iterator;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.scan.result.BatchRawResult;
 import org.carbondata.scan.wrappers.ByteArrayWrapper;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java b/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java
deleted file mode 100644
index cfdc50e..0000000
--- a/core/src/test/java/org/carbondata/core/locks/LocalFileLockTest.java
+++ /dev/null
@@ -1,48 +0,0 @@
-/**
- *
- */
-package org.carbondata.core.locks;
-
-import java.io.File;
-
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-import org.pentaho.di.core.util.Assert;
-
-/**
- * Test class to test the functionality of the local file locking.
- *
- * @author Administrator
- */
-public class LocalFileLockTest {
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Before public void setUp() throws Exception {
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @After public void tearDown() throws Exception {
-  }
-
-  @Test public void testingLocalFileLockingByAcquiring2Locks() {
-
-    LocalFileLock localLock1 =
-        new LocalFileLock((new File(".").getAbsolutePath()) + "/src/test/resources",
-            LockUsage.METADATA_LOCK);
-    Assert.assertTrue(localLock1.lock());
-    LocalFileLock localLock2 =
-        new LocalFileLock((new File(".").getAbsolutePath()) + "/src/test/resources",
-            LockUsage.METADATA_LOCK);
-    Assert.assertTrue(!localLock2.lock());
-
-    Assert.assertTrue(localLock1.unlock());
-    Assert.assertTrue(localLock2.lock());
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java b/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
deleted file mode 100644
index 3bf3993..0000000
--- a/core/src/test/java/org/carbondata/core/locks/ZooKeeperLockingTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/**
- *
- */
-package org.carbondata.core.locks;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.ServerSocket;
-import java.util.Properties;
-
-import org.carbondata.core.util.CarbonProperties;
-
-import mockit.NonStrictExpectations;
-import org.apache.zookeeper.server.ServerConfig;
-import org.apache.zookeeper.server.ZooKeeperServerMain;
-import org.apache.zookeeper.server.quorum.QuorumPeerConfig;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * @author Administrator
- */
-public class ZooKeeperLockingTest {
-
-  int freePort;
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @Before public void setUp() throws Exception {
-    Properties startupProperties = new Properties();
-    startupProperties.setProperty("dataDir", (new File("./target").getAbsolutePath()));
-    startupProperties.setProperty("dataLogDir", (new File("./target").getAbsolutePath()));
-    freePort = findFreePort();
-    startupProperties.setProperty("clientPort", "" + freePort);
-    QuorumPeerConfig quorumConfiguration = new QuorumPeerConfig();
-    try {
-      quorumConfiguration.parseProperties(startupProperties);
-    } catch (Exception e) {
-      throw new RuntimeException(e);
-    }
-
-    final ZooKeeperServerMain zooKeeperServer = new ZooKeeperServerMain();
-    final ServerConfig configuration = new ServerConfig();
-    configuration.readFrom(quorumConfiguration);
-    new Thread() {
-      public void run() {
-        try {
-          zooKeeperServer.runFromConfig(configuration);
-        } catch (IOException e) {
-          System.out.println("ZooKeeper failure");
-        }
-      }
-    }.start();
-  }
-
-  /**
-   * @throws java.lang.Exception
-   */
-  @After public void tearDown() throws Exception {
-  }
-
-  @Test public void testZooKeeperLockingByTryingToAcquire2Locks()
-      throws IllegalArgumentException, IllegalAccessException, NoSuchFieldException,
-      SecurityException {
-
-    final CarbonProperties cp = CarbonProperties.getInstance();
-    new NonStrictExpectations(cp) {
-      {
-        cp.getProperty("/CarbonLocks");
-        result = "/carbontests";
-        cp.getProperty("spark.deploy.zookeeper.url");
-        result = "127.0.0.1:" + freePort;
-      }
-    };
-
-    ZookeeperInit zki = ZookeeperInit.getInstance("127.0.0.1:" + freePort);
-
-    ZooKeeperLocking zkl =
-        new ZooKeeperLocking("D:/carbondata/examples/target/store/default/t3/Metadata",
-            LockUsage.METADATA_LOCK);
-    Assert.assertTrue(zkl.lock());
-
-    ZooKeeperLocking zk2 = new ZooKeeperLocking(
-        "D:/carbondata/examples/target/store/default/t3/Metadata", LockUsage.METADATA_LOCK);
-    Assert.assertTrue(!zk2.lock());
-
-    Assert.assertTrue(zkl.unlock());
-    Assert.assertTrue(zk2.lock());
-    Assert.assertTrue(zk2.unlock());
-  }
-
-  /**
-   * For finding the free port available.
-   *
-   * @return
-   */
-  private static int findFreePort() {
-    ServerSocket socket = null;
-    try {
-      socket = new ServerSocket(0);
-      socket.setReuseAddress(true);
-      int port = socket.getLocalPort();
-      try {
-        socket.close();
-      } catch (IOException e) {
-        // Ignore IOException on close()
-      }
-      return port;
-    } catch (Exception e) {
-      // Ignore
-    } finally {
-      if (socket != null) {
-        try {
-          socket.close();
-        } catch (IOException e) {
-          throw new RuntimeException(e);
-        }
-      }
-    }
-    return 2181;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
index 774ad14..f0f996c 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
@@ -5,9 +5,9 @@ import java.util.ArrayList;
 import java.util.List;
 import java.util.Map;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.core.cache.dictionary.Dictionary;
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.hadoop.readsupport.CarbonReadSupport;
 import org.carbondata.scan.executor.QueryExecutorFactory;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
index 1190a9d..c5a8163 100644
--- a/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
@@ -58,9 +58,9 @@ import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.fileperations.AtomicFileOperations;
-import org.carbondata.core.datastorage.store.fileperations.AtomicFileOperationsImpl;
-import org.carbondata.core.datastorage.store.fileperations.FileWriteOperation;
+import org.carbondata.lcm.fileoperations.AtomicFileOperations;
+import org.carbondata.lcm.fileoperations.AtomicFileOperationsImpl;
+import org.carbondata.lcm.fileoperations.FileWriteOperation;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.load.BlockDetails;
 import org.carbondata.core.load.LoadMetadataDetails;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
index 9ede388..e2cd277 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Set;
 
+import org.carbondata.common.CarbonIterator;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.cache.dictionary.Dictionary;
@@ -35,7 +36,6 @@ import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.scan.executor.QueryExecutor;
 import org.carbondata.scan.executor.QueryExecutorFactory;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
index f83ebf6..6e6f4b9 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
@@ -38,11 +38,11 @@ import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.columnar.ColumnGroupModel;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.DataTypeUtil;
-import org.carbondata.core.vo.ColumnGroupModel;
 import org.carbondata.processing.datatypes.GenericDataType;
 import org.carbondata.processing.merger.exeception.SliceMergerException;
 import org.carbondata.processing.store.CarbonDataFileAttributes;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
index a6615ed..8119d28 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
@@ -40,9 +40,6 @@ import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.fileperations.AtomicFileOperations;
-import org.carbondata.core.datastorage.store.fileperations.AtomicFileOperationsImpl;
-import org.carbondata.core.datastorage.store.fileperations.FileWriteOperation;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
@@ -51,6 +48,9 @@ import org.carbondata.core.load.LoadMetadataDetails;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
+import org.carbondata.lcm.fileoperations.AtomicFileOperations;
+import org.carbondata.lcm.fileoperations.AtomicFileOperationsImpl;
+import org.carbondata.lcm.fileoperations.FileWriteOperation;
 import org.carbondata.processing.api.dataloader.DataLoadModel;
 import org.carbondata.processing.api.dataloader.SchemaInfo;
 import org.carbondata.processing.csvload.DataGraphExecuter;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 3f4dac5..476f25d 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -47,9 +47,9 @@ import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.datastorage.store.impl.FileFactory
-import org.carbondata.core.locks.{CarbonLockFactory, LockUsage}
 import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.carbondata.integration.spark.merger.CompactionType
+import org.carbondata.lcm.locks.{CarbonLockFactory, LockUsage}
 import org.carbondata.lcm.status.SegmentStatusManager
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 import org.carbondata.spark.load._
@@ -1145,6 +1145,7 @@ private[sql] case class AlterTable(
 
 /**
  * Command for the compaction in alter table command
+ *
  * @param alterTableModel
  */
 private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) extends

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
index 3e929a0..dc44f28 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
@@ -47,11 +47,11 @@ import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile
 import org.carbondata.core.datastorage.store.impl.FileFactory
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType
-import org.carbondata.core.locks.ZookeeperInit
 import org.carbondata.core.reader.ThriftReader
 import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
 import org.carbondata.core.writer.ThriftWriter
 import org.carbondata.format.{SchemaEvolutionEntry, TableInfo}
+import org.carbondata.lcm.locks.ZookeeperInit
 import org.carbondata.spark.util.CarbonScalaUtil.CarbonSparkUtil
 
 case class MetaData(var cubesMeta: ArrayBuffer[TableMeta])
@@ -456,10 +456,6 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     dimArray
   }
 
-  def getAggregateTableName(carbonTable: CarbonTable, factTableName: String): String = {
-    CarbonUtil.getNewAggregateTableName(carbonTable.getAggregateTablesName, factTableName)
-  }
-
   /**
    * Shows all schemas which has Database name like
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 87b0952..f770fc7 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -23,14 +23,14 @@ import java.util.concurrent.{Executors, ExecutorService, Future}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ListBuffer
-import scala.util.control.Breaks._
 import scala.util.Random
+import scala.util.control.Breaks._
 
 import org.apache.hadoop.conf.{Configurable, Configuration}
 import org.apache.hadoop.mapreduce.Job
 import org.apache.hadoop.mapreduce.lib.input.FileSplit
 import org.apache.spark.{Logging, Partition, SparkContext, SparkEnv}
-import org.apache.spark.sql.{CarbonEnv, CarbonRelation, SQLContext}
+import org.apache.spark.sql.{CarbonEnv, SQLContext}
 import org.apache.spark.sql.execution.command.{AlterTableModel, CompactionModel, Partitioner}
 import org.apache.spark.util.{FileUtils, SplitUtils}
 
@@ -41,9 +41,9 @@ import org.carbondata.core.carbon.metadata.CarbonMetadata
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
-import org.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
-import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
+import org.carbondata.core.util.CarbonUtil
 import org.carbondata.integration.spark.merger.{CompactionCallable, CompactionType}
+import org.carbondata.lcm.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.carbondata.lcm.status.SegmentStatusManager
 import org.carbondata.processing.util.CarbonDataProcessorUtil
 import org.carbondata.spark._

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
index 84a362b..0de2d1a 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -27,9 +27,9 @@ import org.apache.hadoop.mapreduce.Job
 import org.apache.spark.{Logging, Partition, SparkContext, TaskContext}
 import org.apache.spark.rdd.RDD
 
+import org.carbondata.common.CarbonIterator
 import org.carbondata.common.logging.LogServiceFactory
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo
-import org.carbondata.core.iterator.CarbonIterator
 import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
 import org.carbondata.scan.executor.QueryExecutorFactory
 import org.carbondata.scan.expression.Expression

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java
new file mode 100644
index 0000000..14b39ed
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperations.java
@@ -0,0 +1,33 @@
+/*
+ * 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.carbondata.lcm.fileoperations;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+public interface AtomicFileOperations {
+
+  DataInputStream openForRead() throws IOException;
+
+  void close() throws IOException;
+
+  DataOutputStream openForWrite(FileWriteOperation operation) throws IOException;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java
new file mode 100644
index 0000000..b402eaa
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/fileoperations/AtomicFileOperationsImpl.java
@@ -0,0 +1,87 @@
+/*
+ * 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.carbondata.lcm.fileoperations;
+
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.IOException;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
+import org.carbondata.core.datastorage.store.impl.FileFactory;
+import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
+
+public class AtomicFileOperationsImpl implements AtomicFileOperations {
+
+  private String filePath;
+
+  private FileType fileType;
+
+  private String tempWriteFilePath;
+
+  private DataOutputStream dataOutStream;
+
+  public AtomicFileOperationsImpl(String filePath, FileType fileType) {
+    this.filePath = filePath;
+
+    this.fileType = fileType;
+  }
+
+  @Override public DataInputStream openForRead() throws IOException {
+    return FileFactory.getDataInputStream(filePath, fileType);
+  }
+
+  @Override public DataOutputStream openForWrite(FileWriteOperation operation) throws IOException {
+
+    filePath = filePath.replace("\\", "/");
+
+    tempWriteFilePath = filePath + CarbonCommonConstants.TEMPWRITEFILEEXTENSION;
+
+    if (FileFactory.isFileExist(tempWriteFilePath, fileType)) {
+      FileFactory.getCarbonFile(tempWriteFilePath, fileType).delete();
+    }
+
+    FileFactory.createNewFile(tempWriteFilePath, fileType);
+
+    dataOutStream = FileFactory.getDataOutputStream(tempWriteFilePath, fileType);
+
+    return dataOutStream;
+
+  }
+
+  /* (non-Javadoc)
+   * @see com.huawei.unibi.carbon.datastorage.store.fileperations.AtomicFileOperations#close()
+   */
+  @Override public void close() throws IOException {
+
+    if (null != dataOutStream) {
+      dataOutStream.close();
+
+      CarbonFile tempFile = FileFactory.getCarbonFile(tempWriteFilePath, fileType);
+
+      if (!tempFile.renameForce(filePath)) {
+        throw new IOException("temporary file renaming failed, src="
+            + tempFile.getPath() + ", dest=" + filePath);
+      }
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java b/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java
new file mode 100644
index 0000000..d8832af
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/fileoperations/FileWriteOperation.java
@@ -0,0 +1,25 @@
+/*
+ * 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.carbondata.lcm.fileoperations;
+
+public enum FileWriteOperation {
+
+  APPEND, OVERWRITE
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java b/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java
new file mode 100644
index 0000000..ee76932
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/AbstractCarbonLock.java
@@ -0,0 +1,77 @@
+/*
+ * 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.carbondata.lcm.locks;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonProperties;
+
+/**
+ * This is the abstract class of the lock implementations.This handles the
+ * retrying part of the locking.
+ */
+public abstract class AbstractCarbonLock implements ICarbonLock {
+  private int retryCount;
+
+  private int retryTimeout;
+
+  public abstract boolean lock();
+
+  /**
+   * API for enabling the locking of file with retries.
+   */
+  public boolean lockWithRetries() {
+    try {
+      for (int i = 0; i < retryCount; i++) {
+        if (lock()) {
+          return true;
+        } else {
+          Thread.sleep(retryTimeout * 1000L);
+        }
+      }
+    } catch (InterruptedException e) {
+      return false;
+    }
+    return false;
+  }
+
+  /**
+   * Initializes the retry count and retry timeout.
+   * This will determine how many times to retry to acquire lock and the retry timeout.
+   */
+  protected void initRetry() {
+    String retries = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK);
+    try {
+      retryCount = Integer.parseInt(retries);
+    } catch (NumberFormatException e) {
+      retryCount = CarbonCommonConstants.NUMBER_OF_TRIES_FOR_LOAD_METADATA_LOCK_DEFAULT;
+    }
+
+    String maxTimeout = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK);
+    try {
+      retryTimeout = Integer.parseInt(maxTimeout);
+    } catch (NumberFormatException e) {
+      retryTimeout = CarbonCommonConstants.MAX_TIMEOUT_FOR_LOAD_METADATA_LOCK_DEFAULT;
+    }
+
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/63d3284e/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.java b/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.java
new file mode 100644
index 0000000..8fc1cd5
--- /dev/null
+++ b/processing/src/main/java/org/carbondata/lcm/locks/CarbonLockFactory.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.carbondata.lcm.locks;
+
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonProperties;
+
+/**
+ * This class is a Lock factory class which is used to provide lock objects.
+ * Using this lock object client can request the lock and unlock.
+ */
+public class CarbonLockFactory {
+
+  /**
+   * lockTypeConfigured to check if zookeeper feature is enabled or not for carbon.
+   */
+  private static String lockTypeConfigured;
+
+  static {
+    CarbonLockFactory.updateZooKeeperLockingStatus();
+  }
+
+  /**
+   * This method will determine the lock type.
+   *
+   * @param location
+   * @param lockUsage
+   * @return
+   */
+  public static ICarbonLock getCarbonLockObj(String location, LockUsage lockUsage) {
+    switch (lockTypeConfigured.toUpperCase()) {
+      case CarbonCommonConstants.CARBON_LOCK_TYPE_LOCAL:
+        return new LocalFileLock(location, lockUsage);
+
+      case CarbonCommonConstants.CARBON_LOCK_TYPE_ZOOKEEPER:
+        return new ZooKeeperLocking(location, lockUsage);
+
+      case CarbonCommonConstants.CARBON_LOCK_TYPE_HDFS:
+        return new HdfsFileLock(location, lockUsage);
+
+      default:
+        throw new UnsupportedOperationException("Not supported the lock type");
+    }
+
+  }
+
+  /**
+   * This method will set the zookeeper status whether zookeeper to be used for locking or not.
+   */
+  private static void updateZooKeeperLockingStatus() {
+    lockTypeConfigured = CarbonProperties.getInstance()
+        .getProperty(CarbonCommonConstants.LOCK_TYPE, CarbonCommonConstants.LOCK_TYPE_DEFAULT);
+
+  }
+
+}



[36/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/ExpressionResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/ExpressionResult.java b/core/src/main/java/org/carbondata/query/expression/ExpressionResult.java
deleted file mode 100644
index 3c09c8f..0000000
--- a/core/src/main/java/org/carbondata/query/expression/ExpressionResult.java
+++ /dev/null
@@ -1,413 +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 additiona   l 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.carbondata.query.expression;
-
-import java.math.BigDecimal;
-import java.sql.Timestamp;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.ArrayList;
-import java.util.Date;
-import java.util.List;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class ExpressionResult implements Comparable<ExpressionResult> {
-
-  private static final long serialVersionUID = 1L;
-  protected DataType dataType;
-
-  protected Object value;
-
-  private List<ExpressionResult> expressionResults;
-
-  public ExpressionResult(DataType dataType, Object value) {
-    this.dataType = dataType;
-    this.value = value;
-  }
-
-  public ExpressionResult(List<ExpressionResult> expressionResults) {
-    this.expressionResults = expressionResults;
-  }
-
-  public void set(DataType dataType, Object value) {
-    this.dataType = dataType;
-    this.value = value;
-    this.expressionResults = null;
-  }
-
-  public DataType getDataType() {
-    return dataType;
-  }
-
-  //CHECKSTYLE:OFF Approval No:Approval-V1R2C10_009
-  public Integer getInt() throws FilterUnsupportedException {
-    if (value == null) {
-      return null;
-    }
-    try {
-      switch (this.getDataType()) {
-        case StringType:
-          try {
-            return Integer.parseInt(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterUnsupportedException(e);
-          }
-
-        case IntegerType:
-        case DoubleType:
-
-          if (value instanceof Double) {
-            return ((Double) value).intValue();
-          }
-          return (Integer) value;
-
-        case TimestampType:
-
-          if (value instanceof Timestamp) {
-            return (int) (((Timestamp) value).getTime() % 1000);
-          } else {
-            return (Integer) value;
-          }
-
-        default:
-          throw new FilterUnsupportedException(
-              "Cannot convert" + this.getDataType().name() + " to integer type value");
-      }
-
-    } catch (ClassCastException e) {
-      throw new FilterUnsupportedException(
-          "Cannot convert" + this.getDataType().name() + " to Integer type value");
-    }
-  }
-
-  public String getString() {
-    if (value == null) {
-      return null;
-    }
-    switch (this.getDataType()) {
-      case TimestampType:
-        SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
-            .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-        if (value instanceof Timestamp) {
-          return parser.format((Timestamp) value);
-        } else {
-          return parser.format(new Timestamp((long) value / 1000));
-        }
-
-      default:
-        return value.toString();
-    }
-  }
-
-  public Double getDouble() throws FilterUnsupportedException {
-    if (value == null) {
-      return null;
-    }
-    try {
-      switch (this.getDataType()) {
-        case StringType:
-          try {
-            return Double.parseDouble(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterUnsupportedException(e);
-          }
-
-        case IntegerType:
-          return ((Integer) value).doubleValue();
-        case LongType:
-          return ((Long) value).doubleValue();
-        case DoubleType:
-          return (Double) value;
-        case TimestampType:
-          if (value instanceof Timestamp) {
-            return (double) ((Timestamp) value).getTime() * 1000;
-          } else {
-            return (Double) (value);
-          }
-        default:
-          throw new FilterUnsupportedException(
-              "Cannot convert" + this.getDataType().name() + " to double type value");
-      }
-    } catch (ClassCastException e) {
-      throw new FilterUnsupportedException(
-          "Cannot convert" + this.getDataType().name() + " to Double type value");
-    }
-  }
-  //CHECKSTYLE:ON
-
-  public Long getLong() throws FilterUnsupportedException {
-    if (value == null) {
-      return null;
-    }
-    try {
-      switch (this.getDataType()) {
-        case StringType:
-          try {
-            return Long.parseLong(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterUnsupportedException(e);
-          }
-
-        case IntegerType:
-          return (Long) value;
-        case LongType:
-          return (Long) value;
-        case DoubleType:
-          return (Long) value;
-        case TimestampType:
-          if (value instanceof Timestamp) {
-            return 1000 * ((Timestamp) value).getTime();
-          } else {
-            return (Long) value;
-          }
-        default:
-          throw new FilterUnsupportedException(
-              "Cannot convert" + this.getDataType().name() + " to Long type value");
-      }
-    } catch (ClassCastException e) {
-      throw new FilterUnsupportedException(
-          "Cannot convert" + this.getDataType().name() + " to Long type value");
-    }
-
-  }
-
-  //Add to judge for BigDecimal
-  public BigDecimal getDecimal() throws FilterUnsupportedException {
-    if (value == null) {
-      return null;
-    }
-    try {
-      switch (this.getDataType()) {
-        case StringType:
-          try {
-            return new BigDecimal(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterUnsupportedException(e);
-          }
-
-        case IntegerType:
-          return new BigDecimal((int) value);
-        case LongType:
-          return new BigDecimal((long) value);
-        case DoubleType:
-          return new BigDecimal((double) value);
-        case DecimalType:
-          return new BigDecimal(value.toString());
-        case TimestampType:
-          if (value instanceof Timestamp) {
-            return new BigDecimal(1000 * ((Timestamp) value).getTime());
-          } else {
-            return new BigDecimal((long) value);
-          }
-        default:
-          throw new FilterUnsupportedException(
-              "Cannot convert" + this.getDataType().name() + " to Long type value");
-      }
-    } catch (ClassCastException e) {
-      throw new FilterUnsupportedException(
-          "Cannot convert" + this.getDataType().name() + " to Long type value");
-    }
-
-  }
-
-  public Long getTime() throws FilterUnsupportedException {
-    if (value == null) {
-      return null;
-    }
-    try {
-      switch (this.getDataType()) {
-        case StringType:
-          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-          Date dateToStr;
-          try {
-            dateToStr = parser.parse(value.toString());
-            return dateToStr.getTime() * 1000;
-          } catch (ParseException e) {
-            throw new FilterUnsupportedException(
-                "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
-          }
-        case IntegerType:
-        case LongType:
-          return (Long) value;
-        case DoubleType:
-          return (Long) value;
-        case TimestampType:
-          if (value instanceof Timestamp) {
-            return ((Timestamp) value).getTime() * 1000;
-          } else {
-            return (Long) value;
-          }
-        default:
-          throw new FilterUnsupportedException(
-              "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
-      }
-    } catch (ClassCastException e) {
-      throw new FilterUnsupportedException(
-          "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
-    }
-
-  }
-
-  public Boolean getBoolean() throws FilterUnsupportedException {
-    if (value == null) {
-      return null;
-    }
-    try {
-      switch (this.getDataType()) {
-        case StringType:
-          try {
-            return Boolean.parseBoolean(value.toString());
-          } catch (NumberFormatException e) {
-            throw new FilterUnsupportedException(e);
-          }
-
-        case BooleanType:
-          return Boolean.parseBoolean(value.toString());
-
-        default:
-          throw new FilterUnsupportedException(
-              "Cannot convert" + this.getDataType().name() + " to boolean type value");
-      }
-    } catch (ClassCastException e) {
-      throw new FilterUnsupportedException(
-          "Cannot convert" + this.getDataType().name() + " to Boolean type value");
-    }
-  }
-
-  public List<ExpressionResult> getList() {
-    if (null == expressionResults) {
-      List<ExpressionResult> a = new ArrayList<ExpressionResult>(20);
-      a.add(new ExpressionResult(dataType, value));
-      return a;
-    } else {
-      return expressionResults;
-    }
-  }
-
-  public List<String> getListAsString() {
-    List<String> evaluateResultListFinal = new ArrayList<String>(20);
-    List<ExpressionResult> evaluateResultList = getList();
-    for (ExpressionResult result : evaluateResultList) {
-      if (result.getString() == null) {
-        evaluateResultListFinal.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL);
-        continue;
-      }
-      evaluateResultListFinal.add(result.getString());
-    }
-    return evaluateResultListFinal;
-  }
-
-  @Override public int hashCode() {
-    final int prime = 31;
-    int result = 1;
-    if (null != expressionResults) {
-      result = prime * result + expressionResults.hashCode();
-    } else if (null != value) {
-      result = prime * result + value.toString().hashCode();
-    } else {
-      result = prime * result + "".hashCode();
-    }
-
-    return result;
-  }
-
-  @Override public boolean equals(Object obj) {
-    if (!(obj instanceof ExpressionResult)) {
-      return false;
-    }
-    if (this == obj) {
-      return true;
-    }
-    if (getClass() != obj.getClass()) {
-      return false;
-    }
-    ExpressionResult objToCompare = (ExpressionResult) obj;
-    boolean result = false;
-    if (this.value == objToCompare.value) {
-      return true;
-    }
-    try {
-      switch (this.getDataType()) {
-        case StringType:
-          result = this.getString().equals(objToCompare.getString());
-          break;
-        case IntegerType:
-          result = this.getInt().equals(objToCompare.getInt());
-          break;
-
-        case DoubleType:
-          result = this.getDouble().equals(objToCompare.getDouble());
-          break;
-        case TimestampType:
-          result = this.getLong().equals(objToCompare.getLong());
-          break;
-        default:
-          break;
-      }
-    } catch (FilterUnsupportedException ex) {
-      return false;
-    }
-
-    return result;
-  }
-
-  public boolean isNull() {
-    return value == null;
-  }
-
-  @Override public int compareTo(ExpressionResult o) {
-    try {
-      switch (o.dataType) {
-        case IntegerType:
-        case LongType:
-        case DoubleType:
-
-          Double d1 = this.getDouble();
-          Double d2 = o.getDouble();
-          return d1.compareTo(d2);
-        case DecimalType:
-          java.math.BigDecimal val1 = this.getDecimal();
-          java.math.BigDecimal val2 = o.getDecimal();
-          return val1.compareTo(val2);
-        case TimestampType:
-          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-          Date date1 = null;
-          Date date2 = null;
-          date1 = parser.parse(this.getString());
-          date2 = parser.parse(o.getString());
-          return date1.compareTo(date2);
-        case StringType:
-        default:
-          return this.getString().compareTo(o.getString());
-      }
-    } catch (Exception e) {
-      return -1;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/LeafExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/LeafExpression.java b/core/src/main/java/org/carbondata/query/expression/LeafExpression.java
deleted file mode 100644
index 41093ad..0000000
--- a/core/src/main/java/org/carbondata/query/expression/LeafExpression.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.carbondata.query.expression;
-
-public abstract class LeafExpression extends Expression {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/LiteralExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/LiteralExpression.java b/core/src/main/java/org/carbondata/query/expression/LiteralExpression.java
deleted file mode 100644
index ccad46e..0000000
--- a/core/src/main/java/org/carbondata/query/expression/LiteralExpression.java
+++ /dev/null
@@ -1,68 +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.carbondata.query.expression;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-
-public class LiteralExpression extends LeafExpression {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-  private Object value;
-  private DataType dataType;
-
-  public LiteralExpression(Object value, DataType dataType) {
-    this.value = value;
-    this.dataType = dataType;
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) {
-    ExpressionResult expressionResult = new ExpressionResult(dataType, this.value);
-    return expressionResult;
-  }
-
-  public ExpressionResult getExpressionResult() {
-    ExpressionResult expressionResult = new ExpressionResult(dataType, this.value);
-    return expressionResult;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    // TODO Auto-generated method stub
-    return ExpressionType.LITERAL;
-  }
-
-  @Override public String getString() {
-    // TODO Auto-generated method stub
-    return "LiteralExpression(" + value + ')';
-  }
-
-  /**
-   * getLiteralExpDataType.
-   *
-   * @return
-   */
-  public DataType getLiteralExpDataType() {
-    return dataType;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/UnaryExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/UnaryExpression.java b/core/src/main/java/org/carbondata/query/expression/UnaryExpression.java
deleted file mode 100644
index e8250f9..0000000
--- a/core/src/main/java/org/carbondata/query/expression/UnaryExpression.java
+++ /dev/null
@@ -1,33 +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.carbondata.query.expression;
-
-public abstract class UnaryExpression extends Expression {
-
-  private static final long serialVersionUID = 1L;
-  protected Expression child;
-
-  public UnaryExpression(Expression child) {
-    this.child = child;
-    children.add(child);
-
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/UnknownExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/UnknownExpression.java b/core/src/main/java/org/carbondata/query/expression/UnknownExpression.java
deleted file mode 100644
index c3553d8..0000000
--- a/core/src/main/java/org/carbondata/query/expression/UnknownExpression.java
+++ /dev/null
@@ -1,9 +0,0 @@
-package org.carbondata.query.expression;
-
-import java.util.List;
-
-public abstract class UnknownExpression extends Expression {
-
-  public abstract List<ColumnExpression> getAllColumnList();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/arithmetic/AddExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/arithmetic/AddExpression.java b/core/src/main/java/org/carbondata/query/expression/arithmetic/AddExpression.java
deleted file mode 100644
index bb17da4..0000000
--- a/core/src/main/java/org/carbondata/query/expression/arithmetic/AddExpression.java
+++ /dev/null
@@ -1,81 +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.carbondata.query.expression.arithmetic;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class AddExpression extends BinaryArithmeticExpression {
-  private static final long serialVersionUID = 7999436055420911612L;
-
-  public AddExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult addExprLeftRes = left.evaluate(value);
-    ExpressionResult addExprRightRes = right.evaluate(value);
-    ExpressionResult val1 = addExprLeftRes;
-    ExpressionResult val2 = addExprRightRes;
-    if (addExprLeftRes.isNull() || addExprRightRes.isNull()) {
-      addExprLeftRes.set(addExprLeftRes.getDataType(), null);
-      return addExprLeftRes;
-    }
-
-    if (addExprLeftRes.getDataType() != addExprRightRes.getDataType()) {
-      if (addExprLeftRes.getDataType().getPresedenceOrder() < addExprRightRes.getDataType()
-          .getPresedenceOrder()) {
-        val2 = addExprLeftRes;
-        val1 = addExprRightRes;
-      }
-    }
-    switch (val1.getDataType()) {
-      case StringType:
-      case DoubleType:
-        addExprRightRes.set(DataType.DoubleType, val1.getDouble() + val2.getDouble());
-        break;
-      case IntegerType:
-        addExprRightRes.set(DataType.IntegerType, val1.getInt() + val2.getInt());
-        break;
-      case LongType:
-        addExprRightRes.set(DataType.LongType, val1.getLong() + val2.getLong());
-        break;
-      case DecimalType:
-        addExprRightRes.set(DataType.DecimalType, val1.getDecimal().add(val2.getDecimal()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying Add Expression Filter " + val1.getDataType());
-    }
-    return addExprRightRes;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.ADD;
-  }
-
-  @Override public String getString() {
-    return "Add(" + left.getString() + ',' + right.getString() + ',';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/arithmetic/BinaryArithmeticExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/arithmetic/BinaryArithmeticExpression.java b/core/src/main/java/org/carbondata/query/expression/arithmetic/BinaryArithmeticExpression.java
deleted file mode 100644
index 78f5990..0000000
--- a/core/src/main/java/org/carbondata/query/expression/arithmetic/BinaryArithmeticExpression.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.carbondata.query.expression.arithmetic;
-
-import org.carbondata.query.expression.BinaryExpression;
-import org.carbondata.query.expression.Expression;
-
-public abstract class BinaryArithmeticExpression extends BinaryExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  public BinaryArithmeticExpression(Expression left, Expression right) {
-    super(left, right);
-    // TODO Auto-generated constructor stub
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/arithmetic/DivideExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/arithmetic/DivideExpression.java b/core/src/main/java/org/carbondata/query/expression/arithmetic/DivideExpression.java
deleted file mode 100644
index f1750a0..0000000
--- a/core/src/main/java/org/carbondata/query/expression/arithmetic/DivideExpression.java
+++ /dev/null
@@ -1,81 +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.carbondata.query.expression.arithmetic;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class DivideExpression extends BinaryArithmeticExpression {
-  private static final long serialVersionUID = -7269266926782365612L;
-
-  public DivideExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult divideExprLeftRes = left.evaluate(value);
-    ExpressionResult divideExprRightRes = right.evaluate(value);
-    ExpressionResult val1 = divideExprLeftRes;
-    ExpressionResult val2 = divideExprRightRes;
-    if (divideExprLeftRes.isNull() || divideExprRightRes.isNull()) {
-      divideExprLeftRes.set(divideExprLeftRes.getDataType(), null);
-      return divideExprLeftRes;
-    }
-    if (divideExprLeftRes.getDataType() != divideExprRightRes.getDataType()) {
-      if (divideExprLeftRes.getDataType().getPresedenceOrder() < divideExprRightRes.getDataType()
-          .getPresedenceOrder()) {
-        val2 = divideExprLeftRes;
-        val1 = divideExprRightRes;
-      }
-    }
-    switch (val1.getDataType()) {
-      case StringType:
-      case DoubleType:
-        divideExprRightRes.set(DataType.DoubleType, val1.getDouble() / val2.getDouble());
-        break;
-      case IntegerType:
-        divideExprRightRes.set(DataType.IntegerType, val1.getInt() / val2.getInt());
-        break;
-      case LongType:
-        divideExprRightRes.set(DataType.LongType, val1.getLong() / val2.getLong());
-        break;
-      case DecimalType:
-        divideExprRightRes.set(DataType.DecimalType, val1.getDecimal().divide(val2.getDecimal()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying Add Expression Filter " + divideExprLeftRes
-                .getDataType());
-    }
-    return divideExprRightRes;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.DIVIDE;
-  }
-
-  @Override public String getString() {
-    return "Divide(" + left.getString() + ',' + right.getString() + ')';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/arithmetic/MultiplyExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/arithmetic/MultiplyExpression.java b/core/src/main/java/org/carbondata/query/expression/arithmetic/MultiplyExpression.java
deleted file mode 100644
index 5d02dda..0000000
--- a/core/src/main/java/org/carbondata/query/expression/arithmetic/MultiplyExpression.java
+++ /dev/null
@@ -1,83 +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.carbondata.query.expression.arithmetic;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class MultiplyExpression extends BinaryArithmeticExpression {
-  private static final long serialVersionUID = 1L;
-
-  public MultiplyExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult multiplyExprLeftRes = left.evaluate(value);
-    ExpressionResult multiplyExprRightRes = right.evaluate(value);
-    ExpressionResult val1 = multiplyExprLeftRes;
-    ExpressionResult val2 = multiplyExprRightRes;
-    if (multiplyExprLeftRes.isNull() || multiplyExprRightRes.isNull()) {
-      multiplyExprLeftRes.set(multiplyExprLeftRes.getDataType(), null);
-      return multiplyExprLeftRes;
-    }
-
-    if (multiplyExprLeftRes.getDataType() != multiplyExprRightRes.getDataType()) {
-      if (multiplyExprLeftRes.getDataType().getPresedenceOrder() < multiplyExprRightRes
-          .getDataType().getPresedenceOrder()) {
-        val2 = multiplyExprLeftRes;
-        val1 = multiplyExprRightRes;
-      }
-    }
-    switch (val1.getDataType()) {
-      case StringType:
-      case DoubleType:
-        multiplyExprRightRes.set(DataType.DoubleType, val1.getDouble() * val2.getDouble());
-        break;
-      case IntegerType:
-        multiplyExprRightRes.set(DataType.IntegerType, val1.getInt() * val2.getInt());
-        break;
-      case LongType:
-        multiplyExprRightRes.set(DataType.LongType, val1.getLong() * val2.getLong());
-        break;
-      case DecimalType:
-        multiplyExprRightRes
-            .set(DataType.DecimalType, val1.getDecimal().multiply(val2.getDecimal()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying Add Expression Filter " + multiplyExprLeftRes
-                .getDataType());
-    }
-    return multiplyExprRightRes;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.MULTIPLY;
-  }
-
-  @Override public String getString() {
-    return "Substract(" + left.getString() + ',' + right.getString() + ')';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/arithmetic/SubstractExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/arithmetic/SubstractExpression.java b/core/src/main/java/org/carbondata/query/expression/arithmetic/SubstractExpression.java
deleted file mode 100644
index df1999a..0000000
--- a/core/src/main/java/org/carbondata/query/expression/arithmetic/SubstractExpression.java
+++ /dev/null
@@ -1,83 +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.carbondata.query.expression.arithmetic;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class SubstractExpression extends BinaryArithmeticExpression {
-
-  private static final long serialVersionUID = -8304726440185363102L;
-
-  public SubstractExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult subtractExprLeftRes = left.evaluate(value);
-    ExpressionResult subtractExprRightRes = right.evaluate(value);
-    ExpressionResult val1 = subtractExprLeftRes;
-    ExpressionResult val2 = subtractExprRightRes;
-    if (subtractExprLeftRes.isNull() || subtractExprRightRes.isNull()) {
-      subtractExprLeftRes.set(subtractExprLeftRes.getDataType(), null);
-      return subtractExprLeftRes;
-    }
-    if (subtractExprLeftRes.getDataType() != subtractExprRightRes.getDataType()) {
-      if (subtractExprLeftRes.getDataType().getPresedenceOrder() < subtractExprRightRes
-          .getDataType().getPresedenceOrder()) {
-        val2 = subtractExprLeftRes;
-        val1 = subtractExprRightRes;
-      }
-    }
-    switch (val1.getDataType()) {
-      case StringType:
-      case DoubleType:
-        subtractExprRightRes.set(DataType.DoubleType, val1.getDouble() - val2.getDouble());
-        break;
-      case IntegerType:
-        subtractExprRightRes.set(DataType.IntegerType, val1.getInt() - val2.getInt());
-        break;
-      case LongType:
-        subtractExprRightRes.set(DataType.LongType, val1.getLong() - val2.getLong());
-        break;
-      case DecimalType:
-        subtractExprRightRes
-            .set(DataType.DecimalType, val1.getDecimal().subtract(val2.getDecimal()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying Add Expression Filter " + subtractExprLeftRes
-                .getDataType());
-    }
-    return subtractExprRightRes;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.SUBSTRACT;
-  }
-
-  @Override public String getString() {
-    return "Substract(" + left.getString() + ',' + right.getString() + ')';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/BinaryConditionalExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/BinaryConditionalExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/BinaryConditionalExpression.java
deleted file mode 100644
index 5b347fd..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/BinaryConditionalExpression.java
+++ /dev/null
@@ -1,37 +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.carbondata.query.expression.conditional;
-
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.logical.BinaryLogicalExpression;
-
-public abstract class BinaryConditionalExpression extends BinaryLogicalExpression
-    implements ConditionalExpression {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-
-  public BinaryConditionalExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/ConditionalExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/ConditionalExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/ConditionalExpression.java
deleted file mode 100644
index 12a990d..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/ConditionalExpression.java
+++ /dev/null
@@ -1,43 +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.carbondata.query.expression.conditional;
-
-import java.util.List;
-
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.ExpressionResult;
-
-public interface ConditionalExpression {
-
-  // Will get the column informations involved in the expressions by
-  // traversing the tree
-  List<ColumnExpression> getColumnList();
-
-  boolean isSingleDimension();
-
-  List<ExpressionResult> getLiterals();
-
-  /**
-   * will return the flag of direct dictionary column
-   *
-   * @return
-   */
-  boolean isDirectDictionaryColumns();
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/EqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/EqualToExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/EqualToExpression.java
deleted file mode 100644
index eb180f6..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/EqualToExpression.java
+++ /dev/null
@@ -1,95 +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.carbondata.query.expression.conditional;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class EqualToExpression extends BinaryConditionalExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  public EqualToExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult elRes = left.evaluate(value);
-    ExpressionResult erRes = right.evaluate(value);
-
-    boolean result = false;
-
-    ExpressionResult val1 = elRes;
-    ExpressionResult val2 = erRes;
-
-    if (elRes.isNull() || erRes.isNull()) {
-      result = elRes.isNull() && erRes.isNull();
-      val1.set(DataType.BooleanType, result);
-      return val1;
-    }
-    //default implementation if the data types are different for the resultsets
-    if (elRes.getDataType() != erRes.getDataType()) {
-      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
-        val2 = elRes;
-        val1 = erRes;
-      }
-    }
-
-    // todo: move to util
-    switch (val1.getDataType()) {
-      case StringType:
-        result = val1.getString().equals(val2.getString());
-        break;
-      case IntegerType:
-        result = val1.getInt().equals(val2.getInt());
-        break;
-      case DoubleType:
-        result = val1.getDouble().equals(val2.getDouble());
-        break;
-      case TimestampType:
-        result = val1.getTime().equals(val2.getTime());
-        break;
-      case LongType:
-        result = val1.getLong().equals(val2.getLong());
-        break;
-      case DecimalType:
-        result = val1.getDecimal().compareTo(val2.getDecimal()) == 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
-    }
-    val1.set(DataType.BooleanType, result);
-    return val1;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.EQUALS;
-  }
-
-  @Override public String getString() {
-    return "EqualTo(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanEqualToExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanEqualToExpression.java
deleted file mode 100644
index 5a8cdae..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanEqualToExpression.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements.  See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership.  The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License.  You may obtain a copy of the License at
- *
- *    http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied.  See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.carbondata.query.expression.conditional;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class GreaterThanEqualToExpression extends BinaryConditionalExpression {
-  private static final long serialVersionUID = 4185317066280688984L;
-
-  public GreaterThanEqualToExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult elRes = left.evaluate(value);
-    ExpressionResult erRes = right.evaluate(value);
-    ExpressionResult exprResVal1 = elRes;
-    if (elRes.isNull() || erRes.isNull()) {
-      elRes.set(DataType.BooleanType, false);
-      return elRes;
-    }
-    if (elRes.getDataType() != erRes.getDataType()) {
-      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
-        exprResVal1 = erRes;
-      }
-
-    }
-    boolean result = false;
-    switch (exprResVal1.getDataType()) {
-      case StringType:
-        result = elRes.getString().compareTo(erRes.getString()) >= 0;
-        break;
-      case IntegerType:
-        result = elRes.getInt() >= (erRes.getInt());
-        break;
-      case DoubleType:
-        result = elRes.getDouble() >= (erRes.getDouble());
-        break;
-      case TimestampType:
-        result = elRes.getTime() >= (erRes.getTime());
-        break;
-      case LongType:
-        result = elRes.getLong() >= (erRes.getLong());
-        break;
-      case DecimalType:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) >= 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + exprResVal1.getDataType() + " not supported for the filter expression");
-    }
-    exprResVal1.set(DataType.BooleanType, result);
-    return exprResVal1;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.GREATERTHAN_EQUALTO;
-  }
-
-  @Override public String getString() {
-    return "GreaterThanEqualTo(" + left.getString() + ',' + right.getString() + ')';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanExpression.java
deleted file mode 100644
index 743bfd2..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/GreaterThanExpression.java
+++ /dev/null
@@ -1,88 +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.carbondata.query.expression.conditional;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class GreaterThanExpression extends BinaryConditionalExpression {
-  private static final long serialVersionUID = -5319109756575539219L;
-
-  public GreaterThanExpression(Expression left, Expression right) {
-    super(left, right);
-    // TODO Auto-generated constructor stub
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult exprLeftRes = left.evaluate(value);
-    ExpressionResult exprRightRes = right.evaluate(value);
-    ExpressionResult val1 = exprLeftRes;
-    if (exprLeftRes.isNull() || exprRightRes.isNull()) {
-      exprLeftRes.set(DataType.BooleanType, false);
-      return exprLeftRes;
-    }
-    if (exprLeftRes.getDataType() != exprRightRes.getDataType()) {
-      if (exprLeftRes.getDataType().getPresedenceOrder() < exprRightRes.getDataType()
-          .getPresedenceOrder()) {
-        val1 = exprRightRes;
-      }
-
-    }
-    boolean result = false;
-    switch (val1.getDataType()) {
-      case StringType:
-        result = exprLeftRes.getString().compareTo(exprRightRes.getString()) > 0;
-        break;
-      case DoubleType:
-        result = exprLeftRes.getDouble() > (exprRightRes.getDouble());
-        break;
-      case IntegerType:
-        result = exprLeftRes.getInt() > (exprRightRes.getInt());
-        break;
-      case TimestampType:
-        result = exprLeftRes.getTime() > (exprRightRes.getTime());
-        break;
-      case LongType:
-        result = exprLeftRes.getLong() > (exprRightRes.getLong());
-        break;
-      case DecimalType:
-        result = exprLeftRes.getDecimal().compareTo(exprRightRes.getDecimal()) > 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
-    }
-    val1.set(DataType.BooleanType, result);
-    return val1;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.GREATERTHAN;
-  }
-
-  @Override public String getString() {
-    return "GreaterThan(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/InExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/InExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/InExpression.java
deleted file mode 100644
index cc1feaf..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/InExpression.java
+++ /dev/null
@@ -1,99 +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.carbondata.query.expression.conditional;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class InExpression extends BinaryConditionalExpression {
-  private static final long serialVersionUID = -3149927446694175489L;
-
-  protected transient Set<ExpressionResult> setOfExprResult;
-
-  public InExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult leftRsult = left.evaluate(value);
-
-    if (setOfExprResult == null) {
-      ExpressionResult rightRsult = right.evaluate(value);
-      ExpressionResult val = null;
-      setOfExprResult = new HashSet<ExpressionResult>(10);
-      for (ExpressionResult expressionResVal : rightRsult.getList()) {
-
-        if (leftRsult.getDataType().name().equals(expressionResVal.getDataType().name())) {
-          if (expressionResVal.getDataType().getPresedenceOrder() < leftRsult.getDataType()
-              .getPresedenceOrder()) {
-            val = leftRsult;
-          } else {
-            val = expressionResVal;
-          }
-
-          switch (val.getDataType()) {
-            case StringType:
-              val = new ExpressionResult(val.getDataType(), expressionResVal.getString());
-              break;
-            case IntegerType:
-              val = new ExpressionResult(val.getDataType(), expressionResVal.getInt());
-              break;
-            case DoubleType:
-              val = new ExpressionResult(val.getDataType(), expressionResVal.getDouble());
-              break;
-            case TimestampType:
-              val = new ExpressionResult(val.getDataType(), expressionResVal.getTime());
-              break;
-            case LongType:
-              val = new ExpressionResult(val.getDataType(), expressionResVal.getLong());
-              break;
-            case DecimalType:
-              val = new ExpressionResult(val.getDataType(), expressionResVal.getDecimal());
-              break;
-            default:
-              throw new FilterUnsupportedException(
-                  "DataType: " + val.getDataType() + " not supported for the filter expression");
-          }
-
-        }
-        setOfExprResult.add(val);
-
-      }
-    }
-    leftRsult.set(DataType.BooleanType, setOfExprResult.contains(leftRsult));
-    return leftRsult;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.IN;
-  }
-
-  @Override public String getString() {
-    return "IN(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/LessThanEqualToExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/LessThanEqualToExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/LessThanEqualToExpression.java
deleted file mode 100644
index a40765b..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/LessThanEqualToExpression.java
+++ /dev/null
@@ -1,88 +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.carbondata.query.expression.conditional;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class LessThanEqualToExpression extends BinaryConditionalExpression {
-  private static final long serialVersionUID = 1L;
-
-  public LessThanEqualToExpression(Expression left, Expression right) {
-    super(left, right);
-    // TODO Auto-generated constructor stub
-  }
-
-  public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult elRes = left.evaluate(value);
-    ExpressionResult erRes = right.evaluate(value);
-    ExpressionResult exprResValue1 = elRes;
-    if (elRes.isNull() || erRes.isNull()) {
-      elRes.set(DataType.BooleanType, false);
-      return elRes;
-    }
-    if (elRes.getDataType() != erRes.getDataType()) {
-      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
-        exprResValue1 = erRes;
-      }
-
-    }
-    boolean result = false;
-    switch (exprResValue1.getDataType()) {
-      case StringType:
-        result = elRes.getString().compareTo(erRes.getString()) <= 0;
-        break;
-      case IntegerType:
-        result = elRes.getInt() <= (erRes.getInt());
-        break;
-      case DoubleType:
-        result = elRes.getDouble() <= (erRes.getDouble());
-        break;
-      case TimestampType:
-        result = elRes.getTime() <= (erRes.getTime());
-        break;
-      case LongType:
-        result = elRes.getLong() <= (erRes.getLong());
-        break;
-      case DecimalType:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) <= 0;
-        break;
-      default:
-        throw new FilterUnsupportedException("DataType: " + exprResValue1.getDataType()
-            + " not supported for the filter expression");
-    }
-    exprResValue1.set(DataType.BooleanType, result);
-    return exprResValue1;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    // TODO Auto-generated method stub
-    return ExpressionType.LESSTHAN_EQUALTO;
-  }
-
-  @Override public String getString() {
-    return "LessThanEqualTo(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/LessThanExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/LessThanExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/LessThanExpression.java
deleted file mode 100644
index 8dacc15..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/LessThanExpression.java
+++ /dev/null
@@ -1,90 +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.carbondata.query.expression.conditional;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class LessThanExpression extends BinaryConditionalExpression {
-
-  private static final long serialVersionUID = 6343040416663699924L;
-
-  public LessThanExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult erRes = right.evaluate(value);
-    ExpressionResult elRes = left.evaluate(value);
-
-    ExpressionResult val1 = elRes;
-
-    boolean result = false;
-
-    if (elRes.isNull() || erRes.isNull()) {
-      elRes.set(DataType.BooleanType, false);
-      return elRes;
-    }
-    if (elRes.getDataType() != erRes.getDataType()) {
-      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
-        val1 = erRes;
-      }
-
-    }
-    switch (val1.getDataType()) {
-      case StringType:
-        result = elRes.getString().compareTo(erRes.getString()) < 0;
-        break;
-      case IntegerType:
-        result = elRes.getInt() < (erRes.getInt());
-        break;
-      case DoubleType:
-        result = elRes.getDouble() < (erRes.getDouble());
-        break;
-      case TimestampType:
-        result = elRes.getTime() < (erRes.getTime());
-        break;
-      case LongType:
-        result = elRes.getLong() < (erRes.getLong());
-        break;
-      case DecimalType:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) < 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
-    }
-    val1.set(DataType.BooleanType, result);
-    return val1;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.LESSTHAN;
-  }
-
-  @Override public String getString() {
-    return "LessThan(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/ListExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/ListExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/ListExpression.java
deleted file mode 100644
index 2c7df01..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/ListExpression.java
+++ /dev/null
@@ -1,57 +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.carbondata.query.expression.conditional;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class ListExpression extends Expression {
-  private static final long serialVersionUID = 1L;
-
-  public ListExpression(List<Expression> children) {
-    this.children = children;
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    List<ExpressionResult> listOfExprRes = new ArrayList<ExpressionResult>(10);
-
-    for (Expression expr : children) {
-      listOfExprRes.add(expr.evaluate(value));
-    }
-    return new ExpressionResult(listOfExprRes);
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    // TODO Auto-generated method stub
-    return ExpressionType.LIST;
-  }
-
-  @Override public String getString() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/NotEqualsExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/NotEqualsExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/NotEqualsExpression.java
deleted file mode 100644
index 94957eb..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/NotEqualsExpression.java
+++ /dev/null
@@ -1,93 +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.carbondata.query.expression.conditional;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class NotEqualsExpression extends BinaryConditionalExpression {
-
-  private static final long serialVersionUID = 8684006025540863973L;
-
-  public NotEqualsExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult elRes = left.evaluate(value);
-    ExpressionResult erRes = right.evaluate(value);
-
-    boolean result = false;
-    ExpressionResult val1 = elRes;
-    ExpressionResult val2 = erRes;
-
-    if (elRes.isNull() || erRes.isNull()) {
-      result = elRes.isNull() != erRes.isNull();
-      val1.set(DataType.BooleanType, result);
-      return val1;
-    }
-
-    //default implementation if the data types are different for the resultsets
-    if (elRes.getDataType() != erRes.getDataType()) {
-      //            result = elRes.getString().equals(erRes.getString());
-      if (elRes.getDataType().getPresedenceOrder() < erRes.getDataType().getPresedenceOrder()) {
-        val1 = erRes;
-        val2 = elRes;
-      }
-    }
-    switch (val1.getDataType()) {
-      case StringType:
-        result = !val1.getString().equals(val2.getString());
-        break;
-      case IntegerType:
-        result = val1.getInt().intValue() != val2.getInt().intValue();
-        break;
-      case DoubleType:
-        result = val1.getDouble().doubleValue() != val2.getDouble().doubleValue();
-        break;
-      case TimestampType:
-        result = val1.getTime().longValue() != val2.getTime().longValue();
-        break;
-      case LongType:
-        result = elRes.getLong().longValue() != (erRes.getLong()).longValue();
-        break;
-      case DecimalType:
-        result = elRes.getDecimal().compareTo(erRes.getDecimal()) != 0;
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "DataType: " + val1.getDataType() + " not supported for the filter expression");
-    }
-    val1.set(DataType.BooleanType, result);
-    return val1;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.NOT_EQUALS;
-  }
-
-  @Override public String getString() {
-    return "NotEquals(" + left.getString() + ',' + right.getString() + ')';
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/conditional/NotInExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/conditional/NotInExpression.java b/core/src/main/java/org/carbondata/query/expression/conditional/NotInExpression.java
deleted file mode 100644
index c2ef28f..0000000
--- a/core/src/main/java/org/carbondata/query/expression/conditional/NotInExpression.java
+++ /dev/null
@@ -1,100 +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.carbondata.query.expression.conditional;
-
-import java.util.HashSet;
-import java.util.Set;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class NotInExpression extends BinaryConditionalExpression {
-  private static final long serialVersionUID = -6835841923752118034L;
-  protected transient Set<ExpressionResult> setOfExprResult;
-
-  public NotInExpression(Expression left, Expression right) {
-    super(left, right);
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult leftRsult = left.evaluate(value);
-
-    if (setOfExprResult == null) {
-      ExpressionResult val = null;
-
-      ExpressionResult rightRsult = right.evaluate(value);
-      setOfExprResult = new HashSet<ExpressionResult>(10);
-      for (ExpressionResult exprResVal : rightRsult.getList()) {
-
-        if (leftRsult.getDataType().name().equals(exprResVal.getDataType().name())) {
-          if (exprResVal.getDataType().getPresedenceOrder() < leftRsult.getDataType()
-              .getPresedenceOrder()) {
-            val = leftRsult;
-          } else {
-            val = exprResVal;
-          }
-
-          switch (val.getDataType()) {
-            case StringType:
-              val = new ExpressionResult(val.getDataType(), exprResVal.getString());
-              break;
-            case IntegerType:
-              val = new ExpressionResult(val.getDataType(), exprResVal.getInt());
-              break;
-            case DoubleType:
-              val = new ExpressionResult(val.getDataType(), exprResVal.getDouble());
-              break;
-            case TimestampType:
-              val = new ExpressionResult(val.getDataType(), exprResVal.getTime());
-              break;
-            case LongType:
-              val = new ExpressionResult(val.getDataType(), exprResVal.getLong());
-              break;
-            case DecimalType:
-              val = new ExpressionResult(val.getDataType(), exprResVal.getDecimal());
-              break;
-            default:
-              throw new FilterUnsupportedException(
-                  "DataType: " + val.getDataType() + " not supported for the filter expression");
-          }
-
-        }
-        setOfExprResult.add(val);
-
-      }
-    }
-    leftRsult.set(DataType.BooleanType, !setOfExprResult.contains(leftRsult));
-
-    return leftRsult;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    return ExpressionType.NOT_IN;
-  }
-
-  @Override public String getString() {
-    return "NOT IN(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/exception/FilterUnsupportedException.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/exception/FilterUnsupportedException.java b/core/src/main/java/org/carbondata/query/expression/exception/FilterUnsupportedException.java
deleted file mode 100644
index 62267bf..0000000
--- a/core/src/main/java/org/carbondata/query/expression/exception/FilterUnsupportedException.java
+++ /dev/null
@@ -1,93 +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.carbondata.query.expression.exception;
-
-import java.util.Locale;
-
-public class FilterUnsupportedException extends Exception {
-
-  /**
-   * default serial version ID.
-   */
-  private static final long serialVersionUID = 1L;
-
-  /**
-   * The Error message.
-   */
-  private String msg = "";
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public FilterUnsupportedException(String msg) {
-    super(msg);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public FilterUnsupportedException(String msg, Throwable t) {
-    super(msg, t);
-    this.msg = msg;
-  }
-
-  /**
-   * Constructor
-   *
-   * @param errorCode The error code for this exception.
-   * @param msg       The error message for this exception.
-   */
-  public FilterUnsupportedException(Throwable t) {
-    super(t);
-  }
-
-  /**
-   * This method is used to get the localized message.
-   *
-   * @param locale - A Locale object represents a specific geographical,
-   *               political, or cultural region.
-   * @return - Localized error message.
-   */
-  public String getLocalizedMessage(Locale locale) {
-    return "";
-  }
-
-  /**
-   * getLocalizedMessage
-   */
-  @Override public String getLocalizedMessage() {
-    return super.getLocalizedMessage();
-  }
-
-  /**
-   * getMessage
-   */
-  public String getMessage() {
-    return this.msg;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/logical/AndExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/logical/AndExpression.java b/core/src/main/java/org/carbondata/query/expression/logical/AndExpression.java
deleted file mode 100644
index e74daf2..0000000
--- a/core/src/main/java/org/carbondata/query/expression/logical/AndExpression.java
+++ /dev/null
@@ -1,63 +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.carbondata.query.expression.logical;
-
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public class AndExpression extends BinaryLogicalExpression {
-
-  private static final long serialVersionUID = 1L;
-
-  public AndExpression(Expression left, Expression right) {
-    super(left, right);
-    // TODO Auto-generated constructor stub
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException {
-    ExpressionResult resultLeft = left.evaluate(value);
-    ExpressionResult resultRight = right.evaluate(value);
-    switch (resultLeft.getDataType()) {
-      case BooleanType:
-        resultLeft.set(DataType.BooleanType, (resultLeft.getBoolean() && resultRight.getBoolean()));
-        break;
-      default:
-        throw new FilterUnsupportedException(
-            "Incompatible datatype for applying AND Expression Filter");
-    }
-
-    return resultLeft;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    // TODO Auto-generated method stub
-    return ExpressionType.AND;
-  }
-
-  @Override public String getString() {
-    // TODO Auto-generated method stub
-    return "And(" + left.getString() + ',' + right.getString() + ')';
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/logical/BinaryLogicalExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/logical/BinaryLogicalExpression.java b/core/src/main/java/org/carbondata/query/expression/logical/BinaryLogicalExpression.java
deleted file mode 100644
index 44173f7..0000000
--- a/core/src/main/java/org/carbondata/query/expression/logical/BinaryLogicalExpression.java
+++ /dev/null
@@ -1,127 +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.carbondata.query.expression.logical;
-
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.expression.BinaryExpression;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.ExpressionResult;
-import org.carbondata.query.expression.LiteralExpression;
-
-public abstract class BinaryLogicalExpression extends BinaryExpression {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-
-  public BinaryLogicalExpression(Expression left, Expression right) {
-    super(left, right);
-    // TODO Auto-generated constructor stub
-  }
-
-  public List<ExpressionResult> getLiterals() {
-    List<ExpressionResult> listOfExp =
-        new ArrayList<ExpressionResult>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    getExpressionResultList(this, listOfExp);
-    Collections.sort(listOfExp);
-    return listOfExp;
-  }
-
-  // Will get the column informations involved in the expressions by
-  // traversing the tree
-  public List<ColumnExpression> getColumnList() {
-    // TODO
-    List<ColumnExpression> listOfExp =
-        new ArrayList<ColumnExpression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    getColumnList(this, listOfExp);
-    return listOfExp;
-  }
-
-  private void getColumnList(Expression expression, List<ColumnExpression> lst) {
-    if (expression instanceof ColumnExpression) {
-      ColumnExpression colExp = (ColumnExpression) expression;
-      boolean found = false;
-
-      for (ColumnExpression currentColExp : lst) {
-        if (currentColExp.getColumnName().equals(colExp.getColumnName())) {
-          found = true;
-          colExp.setColIndex(currentColExp.getColIndex());
-          break;
-        }
-      }
-      if (!found) {
-        colExp.setColIndex(lst.size());
-        lst.add(colExp);
-      }
-    }
-    for (Expression child : expression.getChildren()) {
-      getColumnList(child, lst);
-    }
-  }
-
-  public boolean isSingleDimension() {
-    List<ColumnExpression> listOfExp =
-        new ArrayList<ColumnExpression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    getColumnList(this, listOfExp);
-    if (listOfExp.size() == 1 && listOfExp.get(0).isDimension()) {
-      return true;
-    }
-    return false;
-
-  }
-
-  private void getExpressionResultList(Expression binaryConditionalExpression,
-      List<ExpressionResult> listOfExp) {
-    if (binaryConditionalExpression instanceof LiteralExpression) {
-      ExpressionResult colExp =
-          ((LiteralExpression) binaryConditionalExpression).getExpressionResult();
-      listOfExp.add(colExp);
-    }
-    for (Expression child : binaryConditionalExpression.getChildren()) {
-      getExpressionResultList(child, listOfExp);
-    }
-
-  }
-
-  /**
-   * the method will return flag (true or false) depending on the existence of the
-   * direct dictionary columns in conditional expression
-   *
-   * @return the method will return flag (true or false)
-   */
-  public boolean isDirectDictionaryColumns() {
-    List<ColumnExpression> listOfExp =
-        new ArrayList<ColumnExpression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    getColumnList(this, listOfExp);
-    for (ColumnExpression ce : listOfExp) {
-      if (!ce.getCarbonColumn().hasEncoding(Encoding.DICTIONARY)) {
-        return true;
-      }
-    }
-    return false;
-  }
-}


[48/56] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into merge3

Posted by jb...@apache.org.
Merge remote-tracking branch 'carbon_master/master' into merge3

Conflicts:
	core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
	core/src/main/java/org/carbondata/query/aggregator/util/MeasureAggregatorFactory.java
	core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
	core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
	integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
	integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
	integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
	integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
	integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
	integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionNoDictionaryTest.scala


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

Branch: refs/heads/master
Commit: 7972709bcf70a4278535b5195d7c2c3ae934ddc5
Parents: bf7557d 5e2e66f
Author: ravipesala <ra...@gmail.com>
Authored: Mon Jun 20 21:43:22 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Mon Jun 20 21:43:22 2016 +0530

----------------------------------------------------------------------
 .../core/carbon/path/CarbonTablePath.java       |  20 +-
 .../core/constants/CarbonCommonConstants.java   |  27 +
 .../fileperations/AtomicFileOperationsImpl.java |   3 +-
 .../store/filesystem/AbstractDFSCarbonFile.java | 217 +++++++
 .../store/filesystem/HDFSCarbonFile.java        | 249 ++------
 .../store/filesystem/LocalCarbonFile.java       |   1 +
 .../store/filesystem/ViewFSCarbonFile.java      | 126 ++++
 .../store/impl/DFSFileHolderImpl.java           | 183 ++++++
 .../datastorage/store/impl/FileFactory.java     |  36 +-
 .../store/impl/HDFSFileHolderImpl.java          | 186 ------
 .../TimeStampDirectDictionaryGenerator.java     |  17 +-
 .../core/locks/CarbonLockFactory.java           |  10 +-
 .../carbondata/core/locks/ZooKeeperLocking.java |  53 +-
 .../carbondata/core/locks/ZookeeperInit.java    |  64 ++
 .../org/carbondata/core/util/CarbonUtil.java    |  23 +-
 .../aggregator/impl/AvgTimestampAggregator.java | 113 ++++
 .../aggregator/impl/SumTimestampAggregator.java |  89 +++
 .../DirectDictionaryDimensionAggregator.java    | 117 ++++
 .../resolver/ConditionalFilterResolverImpl.java |   9 +-
 .../core/locks/ZooKeeperLockingTest.java        |   9 +-
 .../org/carbondata/examples/CarbonExample.scala |   8 +-
 .../examples/GenerateDictionaryExample.scala    |   2 +-
 .../AbstractDictionaryDecodedReadSupport.java   |   8 +-
 .../carbondata/hadoop/util/SchemaReader.java    |   3 +-
 .../hadoop/ft/CarbonInputMapperTest.java        |  12 +-
 .../hadoop/test/util/StoreCreator.java          |   4 +-
 integration-testcases/pom.xml                   |   2 +-
 .../sql/common/util/CarbonHiveContext.scala     |   4 +-
 .../allqueries/AllDataTypesTestCase2.scala      |   9 +-
 .../allqueries/AllDataTypesTestCase4.scala      |  72 +--
 .../allqueries/AllDataTypesTestCase5.scala      |   9 +-
 .../allqueries/AllDataTypesTestCase6.scala      |  27 +-
 integration/spark/pom.xml                       |   2 +-
 .../spark/merger/CarbonCompactionExecutor.java  |   1 +
 .../spark/merger/CarbonCompactionUtil.java      |  33 --
 .../spark/merger/CompactionCallable.java        |  68 +++
 .../spark/merger/RowResultMerger.java           |  17 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  20 +-
 .../spark/merger/CarbonDataMergerUtil.java      |  28 +-
 .../readsupport/SparkRowReadSupportImpl.java    |   7 +
 .../spark/sql/CarbonCatalystOperators.scala     |   2 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  46 +-
 .../execution/command/carbonTableSchema.scala   |  37 +-
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |  11 +
 .../scala/org/carbondata/spark/KeyVal.scala     |   9 +-
 .../spark/csv/CarbonCsvRelation.scala           | 242 ++++++++
 .../carbondata/spark/csv/DefaultSource.scala    |  94 +--
 .../spark/rdd/CarbonDataLoadRDD.scala           |  20 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        | 245 +++++---
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  27 +-
 .../org/carbondata/spark/rdd/Compactor.scala    |  15 +-
 .../spark/util/GlobalDictionaryUtil.scala       |  53 +-
 .../src/test/resources/InvalidCsvFormatdata.csv |   3 -
 .../spark/src/test/resources/avgTest.csv        |  30 +-
 .../test/resources/compaction/compaction1.csv   |  10 +-
 .../test/resources/compaction/compaction2.csv   |  10 +-
 .../test/resources/compaction/compaction3.csv   |  10 +-
 integration/spark/src/test/resources/data2.csv  |   4 +-
 .../spark/src/test/resources/datasamplenull.csv |   3 +
 .../TestLoadDataWithEmptyArrayColumns.scala     |  11 +-
 .../AllDataTypesTestCaseAggregate.scala         |   4 +
 .../aggquery/AverageQueryTestCase.scala         |   7 +-
 .../AllDataTypesTestCaseAggregate.scala         |  10 +-
 .../testsuite/bigdecimal/TestBigDecimal.scala   |   7 +
 .../spark/testsuite/bigdecimal/TestBigInt.scala |   7 +
 .../DataCompactionCardinalityBoundryTest.scala  |  10 +-
 .../DataCompactionNoDictionaryTest.scala        |  29 +-
 .../datacompaction/DataCompactionTest.scala     |  12 +-
 .../dataload/DefaultSourceTestCase.scala        |  25 +-
 .../TestDataWithDicExcludeAndInclude.scala      |   9 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |   8 +-
 .../dataload/TestLoadTblNameIsKeyword.scala     |  96 +++
 .../dataretention/DataRetentionTestCase.scala   |  11 +-
 .../TimestampDataTypeNullDataTest.scala         |  92 +++
 .../filterexpr/FilterProcessorTestCase.scala    |   9 +
 .../NullMeasureValueTestCaseAggregate.scala     |  14 +-
 .../csvreaderstep/BlockDataHandler.java         | 579 +++++++++----------
 .../processing/csvreaderstep/CsvInput.java      |  10 +-
 .../processing/csvreaderstep/CsvInputMeta.java  |  66 +--
 .../graphgenerator/GraphGenerator.java          |  14 +-
 .../processing/mdkeygen/MDKeyGenStep.java       |   2 +-
 .../sortandgroupby/sortdata/SortDataRows.java   |   2 +-
 .../store/CarbonFactDataHandlerColumnar.java    |  30 +-
 .../store/CarbonFactDataHandlerModel.java       |  22 +
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |  29 +-
 .../FileStoreSurrogateKeyGenForCSV.java         |   2 +-
 .../util/CarbonDataProcessorUtil.java           |  48 +-
 87 files changed, 2620 insertions(+), 1304 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
index d46725f,060adcb..771e5f5
--- a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
@@@ -124,9 -122,10 +127,10 @@@ public class TimeStampDirectDictionaryG
      try {
        dateToStr = timeParser.parse(memberStr);
      } catch (ParseException e) {
 -      LOGGER.error(
 -          "Cannot convert" + TIMESTAMP.toString() + " to Time/Long type value" + e.getMessage());
 +      LOGGER.error("Cannot convert" + memberStr + " to Time/Long type value"
 +          + e.getMessage());
      }
+     //adding +2 to reserve the first cuttOffDiff value for null or empty date
      if (null == dateToStr) {
        return -1;
      } else {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
index 1fb595a,0000000..072d527
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/ConditionalFilterResolverImpl.java
@@@ -1,239 -1,0 +1,242 @@@
 +/*
 + * 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.carbondata.scan.filter.resolver;
 +
 +import java.util.List;
 +import java.util.SortedMap;
 +
 +import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 +import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 +import org.carbondata.core.carbon.metadata.encoder.Encoding;
++import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 +import org.carbondata.scan.executor.exception.QueryExecutionException;
 +import org.carbondata.scan.expression.ColumnExpression;
 +import org.carbondata.scan.expression.DataType;
 +import org.carbondata.scan.expression.Expression;
 +import org.carbondata.scan.expression.conditional.BinaryConditionalExpression;
 +import org.carbondata.scan.expression.conditional.ConditionalExpression;
 +import org.carbondata.scan.expression.exception.FilterUnsupportedException;
 +import org.carbondata.scan.filter.FilterUtil;
 +import org.carbondata.scan.filter.intf.FilterExecuterType;
 +import org.carbondata.scan.filter.resolver.metadata.FilterResolverMetadata;
 +import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 +import org.carbondata.scan.filter.resolver.resolverinfo.visitor.FilterInfoTypeVisitorFactory;
 +
 +public class ConditionalFilterResolverImpl implements FilterResolverIntf {
 +
 +  private static final long serialVersionUID = 1838955268462201691L;
 +  protected Expression exp;
 +  protected boolean isExpressionResolve;
 +  protected boolean isIncludeFilter;
 +  private DimColumnResolvedFilterInfo dimColResolvedFilterInfo;
 +
 +  public ConditionalFilterResolverImpl(Expression exp, boolean isExpressionResolve,
 +      boolean isIncludeFilter) {
 +    this.exp = exp;
 +    this.isExpressionResolve = isExpressionResolve;
 +    this.isIncludeFilter = isIncludeFilter;
 +    this.dimColResolvedFilterInfo = new DimColumnResolvedFilterInfo();
 +  }
 +
 +  /**
 +   * This API will resolve the filter expression and generates the
 +   * dictionaries for executing/evaluating the filter expressions in the
 +   * executer layer.
 +   *
 +   * @throws QueryExecutionException
 +   * @throws FilterUnsupportedException
 +   */
 +  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
 +      throws FilterUnsupportedException {
 +    FilterResolverMetadata metadata = new FilterResolverMetadata();
 +    metadata.setTableIdentifier(absoluteTableIdentifier);
 +    if ((!isExpressionResolve) && exp instanceof BinaryConditionalExpression) {
 +      BinaryConditionalExpression binaryConditionalExpression = (BinaryConditionalExpression) exp;
 +      Expression leftExp = binaryConditionalExpression.getLeft();
 +      Expression rightExp = binaryConditionalExpression.getRight();
 +      if (leftExp instanceof ColumnExpression) {
 +        ColumnExpression columnExpression = (ColumnExpression) leftExp;
 +        metadata.setColumnExpression(columnExpression);
 +        metadata.setExpression(rightExp);
 +        metadata.setIncludeFilter(isIncludeFilter);
 +        // If imei=imei comes in filter condition then we need to
 +        // skip processing of right expression.
 +        // This flow has reached here assuming that this is a single
 +        // column expression.
 +        // we need to check if the other expression contains column
 +        // expression or not in depth.
-         if (FilterUtil.checkIfExpressionContainsColumn(rightExp)||
-             FilterUtil.isExpressionNeedsToResolved(rightExp,isIncludeFilter) &&
-             columnExpression.getDimension().hasEncoding(Encoding.DICTIONARY)){
++        CarbonDimension dimension = columnExpression.getDimension();
++        if (FilterUtil.checkIfExpressionContainsColumn(rightExp)
++            || FilterUtil.isExpressionNeedsToResolved(rightExp, isIncludeFilter) &&
++            dimension.hasEncoding(Encoding.DICTIONARY) && !dimension
++            .hasEncoding(Encoding.DIRECT_DICTIONARY)) {
 +          isExpressionResolve = true;
 +        } else {
 +          //Visitor pattern is been used in this scenario inorder to populate the
 +          // dimColResolvedFilterInfo
 +          //visitable object with filter member values based on the visitor type, currently there
 +          //3 types of visitors custom,direct and no dictionary, all types of visitor populate
 +          //the visitable instance as per its buisness logic which is different for all the
 +          // visitors.
 +          dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
 +              FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
 +              metadata);
 +        }
 +      } else if (rightExp instanceof ColumnExpression) {
 +        ColumnExpression columnExpression = (ColumnExpression) rightExp;
 +        metadata.setColumnExpression(columnExpression);
 +        metadata.setExpression(leftExp);
 +        metadata.setIncludeFilter(isIncludeFilter);
 +        if (columnExpression.getDataType().equals(DataType.TimestampType)) {
 +          isExpressionResolve = true;
 +        } else {
 +          // if imei=imei comes in filter condition then we need to
 +          // skip processing of right expression.
 +          // This flow has reached here assuming that this is a single
 +          // column expression.
 +          // we need to check if the other expression contains column
 +          // expression or not in depth.
 +          if (FilterUtil.checkIfExpressionContainsColumn(leftExp)) {
 +            isExpressionResolve = true;
 +          } else {
 +
 +            dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
 +                FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnExpression),
 +                metadata);
 +
 +          }
 +        }
 +      } else {
 +        isExpressionResolve = true;
 +      }
 +    }
 +    if (isExpressionResolve && exp instanceof ConditionalExpression) {
 +      ConditionalExpression conditionalExpression = (ConditionalExpression) exp;
 +      List<ColumnExpression> columnList = conditionalExpression.getColumnList();
 +      metadata.setColumnExpression(columnList.get(0));
 +      metadata.setExpression(exp);
 +      metadata.setIncludeFilter(isIncludeFilter);
 +      if (!columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) || columnList.get(0)
 +          .getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
 +        dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
 +            FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnList.get(0)), metadata);
 +
 +      } else if (columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY) && !(
 +          columnList.get(0).getDimension().getDataType()
 +              == org.carbondata.core.carbon.metadata.datatype.DataType.STRUCT
 +              || columnList.get(0).getDimension().getDataType()
 +              == org.carbondata.core.carbon.metadata.datatype.DataType.ARRAY)) {
 +        dimColResolvedFilterInfo.setFilterValues(FilterUtil
 +            .getFilterListForAllValues(absoluteTableIdentifier, exp, columnList.get(0),
 +                isIncludeFilter));
 +
 +        dimColResolvedFilterInfo.setColumnIndex(columnList.get(0).getDimension().getOrdinal());
 +        dimColResolvedFilterInfo.setDimension(columnList.get(0).getDimension());
 +      }
 +    }
 +
 +  }
 +
 +  /**
 +   * Left node will not be presentin this scenario
 +   *
 +   * @return left node of type FilterResolverIntf instance
 +   */
 +  public FilterResolverIntf getLeft() {
 +    // TODO Auto-generated method stub
 +    return null;
 +  }
 +
 +  /**
 +   * Right node will not be presentin this scenario
 +   *
 +   * @return left node of type FilterResolverIntf instance
 +   */
 +  @Override public FilterResolverIntf getRight() {
 +    // TODO Auto-generated method stub
 +    return null;
 +  }
 +
 +  /**
 +   * Method will return the DimColumnResolvedFilterInfo instance which consists
 +   * the mapping of the respective dimension and its surrogates involved in
 +   * filter expression.
 +   *
 +   * @return DimColumnResolvedFilterInfo
 +   */
 +  public DimColumnResolvedFilterInfo getDimColResolvedFilterInfo() {
 +    return dimColResolvedFilterInfo;
 +  }
 +
 +  /**
 +   * method will calculates the start key based on the filter surrogates
 +   */
 +  public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
 +      SortedMap<Integer, byte[]> setOfStartKeyByteArray) {
 +    if (null == dimColResolvedFilterInfo.getStarIndexKey()) {
 +      FilterUtil.getStartKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
 +          setOfStartKeyByteArray);
 +    }
 +  }
 +
 +  /**
 +   * method will get the start key based on the filter surrogates
 +   *
 +   * @return end IndexKey
 +   */
 +  @Override public void getEndKey(SegmentProperties segmentProperties,
 +      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
 +      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
 +    if (null == dimColResolvedFilterInfo.getEndIndexKey()) {
 +      try {
 +        FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
 +            absoluteTableIdentifier, endKeys, segmentProperties);
 +        FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
 +            setOfEndKeyByteArray);
 +      } catch (QueryExecutionException e) {
 +        // TODO Auto-generated catch block
 +        e.printStackTrace();
 +      }
 +    }
 +  }
 +
 +  /**
 +   * Method will return the executer type for particular conditional resolver
 +   * basically two types of executers will be formed for the conditional query.
 +   *
 +   * @return the filter executer type
 +   */
 +  @Override public FilterExecuterType getFilterExecuterType() {
 +    switch (exp.getFilterExpressionType()) {
 +      case NOT_EQUALS:
 +      case NOT_IN:
 +        return FilterExecuterType.EXCLUDE;
 +
 +      default:
 +        return FilterExecuterType.INCLUDE;
 +    }
 +
 +  }
 +
 +  @Override public Expression getFilterExpression() {
 +    // TODO Auto-generated method stub
 +    return exp;
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------
diff --cc examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
index a549409,2d37bda..db05a51
--- a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
@@@ -63,9 -63,9 +63,9 @@@ object GenerateDictionaryExample 
      val tableName = carbonTableIdentifier.getTableName
      val carbonRelation = CarbonEnv.getInstance(carbonContext).carbonCatalog.
        lookupRelation1(Option(dataBaseName),
 -        tableName, None) (carbonContext).asInstanceOf[CarbonRelation]
 +        tableName) (carbonContext).asInstanceOf[CarbonRelation]
      val carbonTable = carbonRelation.cubeMeta.carbonTable
-     val dimensions = carbonTable.getDimensionByTableName(tableName)
+     val dimensions = carbonTable.getDimensionByTableName(tableName.toLowerCase())
        .toArray.map(_.asInstanceOf[CarbonDimension])
      // scalastyle:off println
      // print dictionary information

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputMapperTest.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
----------------------------------------------------------------------
diff --cc integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
index 88ba722,14824ce..da328a1
--- a/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
+++ b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
@@@ -113,9 -93,9 +119,10 @@@ class AllDataTypesTestCase2 extends Que
    }
  
    override def afterAll {
+     //CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
      try {
        sql("drop cube Carbon_automation_test2")
 +      sql("drop table Carbon_automation_hive2")
      } catch {
        case e: Exception => print("ERROR: DROP Carbon_automation_test2 ")
      }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/pom.xml
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --cc integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index 85a3418,48b3668..9cf0ea7
--- a/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@@ -3,9 -3,11 +3,11 @@@ package org.carbondata.spark.readsuppor
  import java.sql.Timestamp;
  
  import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+ import org.carbondata.core.carbon.metadata.datatype.DataType;
+ import org.carbondata.core.carbon.metadata.encoder.Encoding;
  import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
  import org.carbondata.hadoop.readsupport.impl.AbstractDictionaryDecodedReadSupport;
 -import org.carbondata.query.carbon.util.DataTypeUtil;
 +import org.carbondata.scan.util.DataTypeUtil;
  
  import org.apache.spark.sql.Row;
  import org.apache.spark.sql.catalyst.expressions.GenericRow;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index e36d148,655141c..3f4dac5
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@@ -25,9 -25,10 +25,11 @@@ import java.util.UUI
  import scala.collection.JavaConverters._
  import scala.collection.mutable.ArrayBuffer
  import scala.language.implicitConversions
+ import scala.util.Random
  
+ import org.apache.spark.SparkEnv
  import org.apache.spark.sql._
 +import org.apache.spark.sql.catalyst.TableIdentifier
  import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
  import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
  import org.apache.spark.sql.hive.HiveContext
@@@ -1395,16 -1412,15 +1398,16 @@@ private[sql] case class LoadTable
  
    def run(sqlContext: SQLContext): Seq[Row] = {
  
 -    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 +    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 +    val identifier = TableIdentifier(tableName, Option(dbName))
      if (isOverwriteExist) {
 -      sys.error("Overwrite is not supported for carbon table with " + schemaName + "." + tableName)
 +      sys.error("Overwrite is not supported for carbon table with " + dbName + "." + tableName)
      }
      if (null == org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance
 -      .getCarbonTable(schemaName + "_" + tableName)) {
 -      logError("Data loading failed. table not found: " + schemaName + "." + tableName)
 -      LOGGER.audit("Data loading failed. table not found: " + schemaName + "." + tableName)
 -      sys.error("Data loading failed. table not found: " + schemaName + "." + tableName)
 +      .getCarbonTable(dbName + "_" + tableName)) {
-       logError("Data loading failed. table not found: " + dbName + "_" + tableName)
-       LOGGER.audit("Data loading failed. table not found: " + dbName + "_" + tableName)
-       sys.error("Data loading failed. table not found: " + dbName + "_" + tableName)
++      logError("Data loading failed. table not found: " + dbName + "." + tableName)
++      LOGGER.audit("Data loading failed. table not found: " + dbName + "." + tableName)
++      sys.error("Data loading failed. table not found: " + dbName + "." + tableName)
      }
      CarbonProperties.getInstance().addProperty("zookeeper.enable.lock", "false")
      val carbonLock = CarbonLockFactory.getCarbonLockObj(org.carbondata.core.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
index cb87818,66bb0c0..52103be
--- a/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/KeyVal.scala
@@@ -26,30 -26,33 +26,29 @@@
  package org.carbondata.spark
  
  import org.carbondata.core.load.LoadMetadataDetails
 -import org.carbondata.query.carbon.result.BatchRawResult
 -import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
 -
 -trait KeyVal[K, V] extends Serializable {
 -  def getKey(key: CarbonKey, value: CarbonValue): (K, V)
  
 +trait Value[V] extends Serializable {
 +  def getValue(value: Array[Object]): V
  }
  
 -class KeyValImpl extends KeyVal[CarbonKey, CarbonValue] {
 -  override def getKey(key: CarbonKey, value: CarbonValue): (CarbonKey, CarbonValue) = (key, value)
 +class ValueImpl extends Value[Array[Object]] {
 +  override def getValue(value: Array[Object]): Array[Object] = value
  }
  
 -trait RawKeyVal[K, V] extends Serializable {
 -  def getKey(key: BatchRawResult, value: Any): (K, V)
 -
 +trait RawValue[V] extends Serializable {
 +  def getValue(value: Array[Any]): V
  }
  
 -class RawKeyValImpl extends RawKeyVal[BatchRawResult, Any] {
 -  override def getKey(key: BatchRawResult, value: Any): (BatchRawResult, Any) = (key, value)
 +class RawValueImpl extends RawValue[Array[Any]] {
 +  override def getValue(value: Array[Any]): Array[Any] = value
  }
  
- trait Result[K, V] extends Serializable {
-   def getKey(key: Int, value: LoadMetadataDetails): (K, V)
- 
+ trait DataLoadResult[K, V] extends Serializable {
+   def getKey(key: String, value: LoadMetadataDetails): (K, V)
  }
  
- class ResultImpl extends Result[Int, LoadMetadataDetails] {
-   override def getKey(key: Int, value: LoadMetadataDetails): (Int, LoadMetadataDetails) = {
+ class DataLoadResultImpl extends DataLoadResult[String, LoadMetadataDetails] {
+   override def getKey(key: String, value: LoadMetadataDetails): (String, LoadMetadataDetails) = {
      (key, value)
    }
  }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index af2271f,839a02e..87b0952
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@@ -40,11 -41,12 +41,11 @@@ import org.carbondata.core.carbon.metad
  import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
  import org.carbondata.core.constants.CarbonCommonConstants
  import org.carbondata.core.load.{BlockDetails, LoadMetadataDetails}
- import org.carbondata.core.locks.{CarbonLockFactory, LockUsage}
- import org.carbondata.core.util.CarbonUtil
- import org.carbondata.integration.spark.merger.CompactionType
+ import org.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
+ import org.carbondata.core.util.{CarbonProperties, CarbonUtil}
+ import org.carbondata.integration.spark.merger.{CompactionCallable, CompactionType}
  import org.carbondata.lcm.status.SegmentStatusManager
  import org.carbondata.processing.util.CarbonDataProcessorUtil
 -import org.carbondata.query.scanner.impl.{CarbonKey, CarbonValue}
  import org.carbondata.spark._
  import org.carbondata.spark.load._
  import org.carbondata.spark.merger.CarbonDataMergerUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
index e18b64d,c418216..c1361f5
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
@@@ -34,9 -34,10 +34,9 @@@ import org.carbondata.core.carbon.metad
  import org.carbondata.core.constants.CarbonCommonConstants
  import org.carbondata.core.util.CarbonProperties
  import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
- import org.carbondata.integration.spark.merger.{CarbonCompactionExecutor, CarbonCompactionUtil,
- RowResultMerger}
+ import org.carbondata.integration.spark.merger.{CarbonCompactionExecutor, CarbonCompactionUtil, RowResultMerger}
+ import org.carbondata.processing.util.CarbonDataProcessorUtil
 -import org.carbondata.query.carbon.result.RowResult
 -import org.carbondata.query.carbon.result.iterator.RawResultIterator
 +import org.carbondata.scan.result.iterator.RawResultIterator
  import org.carbondata.spark.MergeResult
  import org.carbondata.spark.load.{CarbonLoaderUtil, CarbonLoadModel}
  import org.carbondata.spark.merger.CarbonDataMergerUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7972709b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
index c2a2277,3b389a6..b6bda0c
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
@@@ -37,18 -39,18 +39,22 @@@ class AllDataTypesTestCaseAggregate ext
  
      val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
        .getCanonicalPath
-     CarbonProperties.getInstance().addProperty("carbon.direct.surrogate","false")
+ 
      sql("create cube Carbon_automation_test dimensions(imei string,deviceInformationId integer,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Active
 _phonePADPartitionedVersions string, Latest_YEAR integer, Latest_MONTH integer, Latest_DAY integer, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string)  measures(gamePointId integer,contractNumber integer) OPTIONS (PARTITIONER [CLASS = 'org.carbondata.spark.partition.api.impl.SampleDataPartitionerImpl' ,COLUMNS= (imei) , PARTITION_COUNT=2] )");
+     CarbonProperties.getInstance()
+       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT)
      sql("LOAD DATA FACT FROM'"+currentDirectory+"/src/test/resources/100_olap.csv' INTO Cube Carbon_automation_test partitionData(DELIMITER ',' ,QUOTECHAR '\"', FILEHEADER 'imei,deviceInformationId,MAC,deviceColor,device_backColor,modelId,marketName,AMSize,ROMSize,CUPAudit,CPIClocked,series,productionDate,bomCode,internalModels,deliveryTime,channelsId,channelsName,deliveryAreaId,deliveryCountry,deliveryProvince,deliveryCity,deliveryDistrict,deliveryStreet,oxSingleNumber,contractNumber,ActiveCheckTime,ActiveAreaId,ActiveCountry,ActiveProvince,Activecity,ActiveDistrict,ActiveStreet,ActiveOperatorId,Active_releaseId,Active_EMUIVersion,Active_operaSysVersion,Active_BacVerNumber,Active_BacFlashVer,Active_webUIVersion,Active_webUITypeCarrVer,Active_webTypeDataVerNumber,Active_operatorsVersion,Active_phonePADPartitionedVersions,Latest_YEAR,Latest_MONTH,Latest_DAY,Latest_HOUR,Latest_areaId,Latest_country,Latest_province,Latest_city,Latest_district,Latest_street,Latest_releaseId,Latest_EMUI
 Version,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber,Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')");
 +    sql("create table if not exists Carbon_automation_hive (imei string,deviceInformationId int,MAC string,deviceColor string,device_backColor string,modelId string,marketName string,AMSize string,ROMSize string,CUPAudit string,CPIClocked string,series string,productionDate timestamp,bomCode string,internalModels string, deliveryTime string, channelsId string, channelsName string , deliveryAreaId string, deliveryCountry string, deliveryProvince string, deliveryCity string,deliveryDistrict string, deliveryStreet string, oxSingleNumber string, ActiveCheckTime string, ActiveAreaId string, ActiveCountry string, ActiveProvince string, Activecity string, ActiveDistrict string, ActiveStreet string, ActiveOperatorId string, Active_releaseId string, Active_EMUIVersion string, Active_operaSysVersion string, Active_BacVerNumber string, Active_BacFlashVer string, Active_webUIVersion string, Active_webUITypeCarrVer string,Active_webTypeDataVerNumber string, Active_operatorsVersion string, Activ
 e_phonePADPartitionedVersions string, Latest_YEAR int, Latest_MONTH int, Latest_DAY int, Latest_HOUR string, Latest_areaId string, Latest_country string, Latest_province string, Latest_city string, Latest_district string, Latest_street string, Latest_releaseId string, Latest_EMUIVersion string, Latest_operaSysVersion string, Latest_BacVerNumber string, Latest_BacFlashVer string, Latest_webUIVersion string, Latest_webUITypeCarrVer string, Latest_webTypeDataVerNumber string, Latest_operatorsVersion string, Latest_phonePADPartitionedVersions string, Latest_operatorId string, gamePointDescription string, gamePointId int,contractNumber int) row format delimited fields terminated by ','");
 +    sql("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO table Carbon_automation_hive ");
  
    }
  
    override def afterAll {
      sql("drop cube Carbon_automation_test")
 +    sql("drop cube Carbon_automation_hive")
 +
+     CarbonProperties.getInstance()
+       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
    }
  
    //Test-22



[37/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
deleted file mode 100644
index 458a4c8..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/DetailQueryResultIterator.java
+++ /dev/null
@@ -1,84 +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.carbondata.query.carbon.result.iterator;
-
-import java.util.List;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-
-/**
- * In case of detail query we cannot keep all the records in memory so for
- * executing that query are returning a iterator over block and every time next
- * call will come it will execute the block and return the result
- */
-public class DetailQueryResultIterator extends AbstractDetailQueryResultIterator {
-
-  /**
-   * to prepare the result
-   */
-  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
-
-  private ExecutorService execService = Executors.newFixedThreadPool(1);
-
-  private Future<BatchResult> future;
-
-  public DetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel,
-      QueryResultPreparator queryResultPreparator) {
-    super(infos, queryModel);
-    this.queryResultPreparator = queryResultPreparator;
-  }
-
-  @Override public BatchResult next() {
-    BatchResult result;
-    try {
-      if (future == null) {
-        future = execute();
-      }
-      result = future.get();
-      nextBatch = false;
-      if (hasNext()) {
-        nextBatch = true;
-        future = execute();
-      } else {
-        fileReader.finish();
-      }
-    } catch (Exception ex) {
-      fileReader.finish();
-      throw new RuntimeException(ex.getCause().getMessage());
-    }
-    return result;
-  }
-
-  private Future<BatchResult> execute() {
-    return execService.submit(new Callable<BatchResult>() {
-      @Override public BatchResult call() throws QueryExecutionException {
-        return queryResultPreparator.prepareQueryResult(dataBlockIterator.next());
-      }
-    });
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java
deleted file mode 100644
index c99ee15..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/MemoryBasedResultIterator.java
+++ /dev/null
@@ -1,51 +0,0 @@
-package org.carbondata.query.carbon.result.iterator;
-
-import java.util.NoSuchElementException;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Memory based result iterator
- */
-public class MemoryBasedResultIterator extends CarbonIterator<Result> {
-
-  /**
-   * query result
-   */
-  private Result result;
-
-  /**
-   * to check any more
-   */
-  private boolean hasNext = true;
-
-  public MemoryBasedResultIterator(Result result) {
-    this.result = result;
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements. (In other words,
-   * returns {@code true} if {@link #next} would return an element rather than
-   * throwing an exception.)
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return hasNext;
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Result next() {
-    if (hasNext) {
-      hasNext = false;
-      return result;
-    }
-    throw new NoSuchElementException();
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java
deleted file mode 100644
index 27114f2..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/RawResultIterator.java
+++ /dev/null
@@ -1,167 +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.carbondata.query.carbon.result.iterator;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.carbon.result.BatchRawResult;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * This is a wrapper iterator over the detail raw query iterator.
- * This iterator will handle the processing of the raw rows.
- * This will handle the batch results and will iterate on the batches and give single row.
- */
-public class RawResultIterator extends CarbonIterator<Object[]> {
-
-  private final SegmentProperties sourceSegProperties;
-
-  private final SegmentProperties destinationSegProperties;
-  /**
-   * Iterator of the Batch raw result.
-   */
-  private CarbonIterator<BatchRawResult> detailRawQueryResultIterator;
-
-  /**
-   * Counter to maintain the row counter.
-   */
-  private int counter = 0;
-
-  private Object[] currentConveretedRawRow = null;
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(RawResultIterator.class.getName());
-
-  /**
-   * batch of the result.
-   */
-  private BatchRawResult batch;
-
-  public RawResultIterator(CarbonIterator<BatchRawResult> detailRawQueryResultIterator,
-      SegmentProperties sourceSegProperties, SegmentProperties destinationSegProperties) {
-    this.detailRawQueryResultIterator = detailRawQueryResultIterator;
-    this.sourceSegProperties = sourceSegProperties;
-    this.destinationSegProperties = destinationSegProperties;
-  }
-
-  @Override public boolean hasNext() {
-
-    if (null == batch || checkIfBatchIsProcessedCompletely(batch)) {
-      if (detailRawQueryResultIterator.hasNext()) {
-        batch = detailRawQueryResultIterator.next();
-        counter = 0; // batch changed so reset the counter.
-      } else {
-        return false;
-      }
-    }
-
-    if (!checkIfBatchIsProcessedCompletely(batch)) {
-      return true;
-    } else {
-      return false;
-    }
-  }
-
-  @Override public Object[] next() {
-    if (null == batch) { // for 1st time
-      batch = detailRawQueryResultIterator.next();
-    }
-    if (!checkIfBatchIsProcessedCompletely(batch)) {
-      try {
-        if(null != currentConveretedRawRow){
-          counter++;
-          Object[] currentConveretedRawRowTemp = this.currentConveretedRawRow;
-          currentConveretedRawRow = null;
-          return currentConveretedRawRowTemp;
-        }
-        return convertRow(batch.getRawRow(counter++));
-      } catch (KeyGenException e) {
-        LOGGER.error(e.getMessage());
-        return null;
-      }
-    } else { // completed one batch.
-      batch = detailRawQueryResultIterator.next();
-      counter = 0;
-    }
-    try {
-      if(null != currentConveretedRawRow){
-        counter++;
-        Object[] currentConveretedRawRowTemp = this.currentConveretedRawRow;
-        currentConveretedRawRow = null;
-        return currentConveretedRawRowTemp;
-      }
-
-      return convertRow(batch.getRawRow(counter++));
-    } catch (KeyGenException e) {
-      LOGGER.error(e.getMessage());
-      return null;
-    }
-
-  }
-
-  /**
-   * for fetching the row with out incrementing counter.
-   * @return
-   */
-  public Object[] fetchConverted() throws KeyGenException {
-    if(null != currentConveretedRawRow){
-      return currentConveretedRawRow;
-    }
-    if(hasNext())
-    {
-      Object[] rawRow = batch.getRawRow(counter);
-      currentConveretedRawRow = convertRow(rawRow);;
-      return currentConveretedRawRow;
-    }
-    else
-    {
-      return null;
-    }
-  }
-
-  private Object[] convertRow(Object[] rawRow) throws KeyGenException {
-    byte[] dims = ((ByteArrayWrapper) rawRow[0]).getDictionaryKey();
-    long[] keyArray = sourceSegProperties.getDimensionKeyGenerator().getKeyArray(dims);
-    byte[] covertedBytes =
-        destinationSegProperties.getDimensionKeyGenerator().generateKey(keyArray);
-    ((ByteArrayWrapper) rawRow[0]).setDictionaryKey(covertedBytes);
-    return rawRow;
-  }
-
-  /**
-   * To check if the batch is processed completely
-   * @param batch
-   * @return
-   */
-  private boolean checkIfBatchIsProcessedCompletely(BatchRawResult batch){
-    if(counter < batch.getSize())
-    {
-      return false;
-    }
-    else{
-      return true;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
deleted file mode 100644
index fbf3074..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/QueryResultPreparator.java
+++ /dev/null
@@ -1,10 +0,0 @@
-package org.carbondata.query.carbon.result.preparator;
-
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.Result;
-
-public interface QueryResultPreparator<K, V> {
-
-  public BatchResult prepareQueryResult(Result<K, V> scannedResult);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
deleted file mode 100644
index ad5b4c5..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/AbstractQueryResultPreparator.java
+++ /dev/null
@@ -1,87 +0,0 @@
-package org.carbondata.query.carbon.result.preparator.impl;
-
-import java.util.List;
-
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-
-public abstract class AbstractQueryResultPreparator<K, V> implements QueryResultPreparator<K, V> {
-
-  /**
-   * query properties
-   */
-  protected QueryExecutorProperties queryExecuterProperties;
-
-  /**
-   * query model
-   */
-  protected QueryModel queryModel;
-
-  public AbstractQueryResultPreparator(QueryExecutorProperties executerProperties,
-      QueryModel queryModel) {
-    this.queryExecuterProperties = executerProperties;
-    this.queryModel = queryModel;
-  }
-
-  protected void fillDimensionData(Object[][] convertedResult, List<QueryDimension> queryDimensions,
-      int dimensionCount, Object[] row, int rowIndex) {
-    QueryDimension queryDimension;
-    for (int i = 0; i < dimensionCount; i++) {
-      queryDimension = queryDimensions.get(i);
-      if (!CarbonUtil
-          .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DICTIONARY)) {
-        row[queryDimension.getQueryOrder()] = convertedResult[i][rowIndex];
-      } else if (CarbonUtil
-          .hasEncoding(queryDimension.getDimension().getEncoder(), Encoding.DIRECT_DICTIONARY)) {
-        DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
-            .getDirectDictionaryGenerator(queryDimension.getDimension().getDataType());
-        row[queryDimension.getQueryOrder()] = directDictionaryGenerator
-            .getValueFromSurrogate((Integer) convertedResult[i][rowIndex]);
-      } else {
-        if (queryExecuterProperties.sortDimIndexes[i] == 1) {
-          row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
-              queryExecuterProperties.columnToDictionayMapping
-                  .get(queryDimension.getDimension().getColumnId())
-                  .getDictionaryValueFromSortedIndex((Integer) convertedResult[i][rowIndex]),
-              queryDimension.getDimension().getDataType());
-        } else {
-          row[queryDimension.getQueryOrder()] = DataTypeUtil.getDataBasedOnDataType(
-              queryExecuterProperties.columnToDictionayMapping
-                  .get(queryDimension.getDimension().getColumnId())
-                  .getDictionaryValueForKey((Integer) convertedResult[i][rowIndex]),
-              queryDimension.getDimension().getDataType());
-        }
-      }
-    }
-  }
-
-  protected Object[][] encodeToRows(Object[][] data) {
-    if (data.length == 0) {
-      return data;
-    }
-    Object[][] rData = new Object[data[0].length][data.length];
-    int len = data.length;
-    for (int i = 0; i < rData.length; i++) {
-      for (int j = 0; j < len; j++) {
-        rData[i][j] = data[j][i];
-      }
-    }
-    return rData;
-  }
-
-  protected BatchResult getEmptyChunkResult(int size) {
-    Object[][] row = new Object[size][1];
-    BatchResult chunkResult = new BatchResult();
-    chunkResult.setRows(row);
-    return chunkResult;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
deleted file mode 100644
index 712894a..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/DetailQueryResultPreparatorImpl.java
+++ /dev/null
@@ -1,139 +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.carbondata.query.carbon.result.preparator.impl;
-
-import java.nio.charset.Charset;
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * Below class will be used to get the result by converting to actual data
- * Actual data conversion can be converting the surrogate key to actual data
- *
- * @TODO there are many things in class which is very confusing, need to check
- * why it was handled like that and how we can handle that in a better
- * way.Need to revisit this class. IF aggregation is push down to spark
- * layer and if we can process the data in byte array format then this
- * class wont be useful so in future we can delete this class.
- * @TODO need to expose one interface which will return the result based on required type
- * for example its implementation case return converted result or directly result with out
- * converting to actual value
- */
-public class DetailQueryResultPreparatorImpl
-    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, Object> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DetailQueryResultPreparatorImpl.class.getName());
-
-  public DetailQueryResultPreparatorImpl(QueryExecutorProperties executerProperties,
-      QueryModel queryModel) {
-    super(executerProperties, queryModel);
-  }
-
-  @Override public BatchResult prepareQueryResult(
-      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
-    if ((null == scannedResult || scannedResult.size() < 1)) {
-      return new BatchResult();
-    }
-    List<QueryDimension> queryDimension = queryModel.getQueryDimension();
-    int dimensionCount = queryDimension.size();
-    int totalNumberOfColumn = dimensionCount + queryExecuterProperties.measureDataTypes.length;
-    Object[][] resultData = new Object[scannedResult.size()][totalNumberOfColumn];
-    if (!queryExecuterProperties.isFunctionQuery && totalNumberOfColumn == 0
-        && scannedResult.size() > 0) {
-      return getEmptyChunkResult(scannedResult.size());
-    }
-    int currentRow = 0;
-    long[] surrogateResult = null;
-    int noDictionaryColumnIndex = 0;
-    ByteArrayWrapper key = null;
-    Object[] value = null;
-    while (scannedResult.hasNext()) {
-      key = scannedResult.getKey();
-      value = scannedResult.getValue();
-      if (key != null) {
-        surrogateResult = queryExecuterProperties.keyStructureInfo.getKeyGenerator()
-            .getKeyArray(key.getDictionaryKey(),
-                queryExecuterProperties.keyStructureInfo.getMaskedBytes());
-        for (int i = 0; i < dimensionCount; i++) {
-          if (!CarbonUtil.hasEncoding(queryDimension.get(i).getDimension().getEncoder(),
-              Encoding.DICTIONARY)) {
-            resultData[currentRow][i] = DataTypeUtil.getDataBasedOnDataType(
-                new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++),
-                    Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
-                queryDimension.get(i).getDimension().getDataType());
-          } else {
-            resultData[currentRow][i] =
-                (int) surrogateResult[queryDimension.get(i).getDimension().getKeyOrdinal()];
-          }
-        }
-      }
-      if (value != null) {
-        System.arraycopy(value, 0, resultData[currentRow], dimensionCount,
-            queryExecuterProperties.measureDataTypes.length);
-      }
-      currentRow++;
-      noDictionaryColumnIndex = 0;
-    }
-    if (resultData.length > 0) {
-      resultData = encodeToRows(resultData);
-    }
-    return getResult(queryModel, resultData);
-  }
-
-  private BatchResult getResult(QueryModel queryModel, Object[][] convertedResult) {
-
-    int rowSize = convertedResult[0].length;
-    Object[][] rows = new Object[rowSize][];
-    List<QueryDimension> queryDimensions = queryModel.getQueryDimension();
-    int dimensionCount = queryDimensions.size();
-    int msrCount = queryExecuterProperties.measureDataTypes.length;
-    Object[] row;
-    for (int rowIndex = 0; rowIndex < rowSize; rowIndex++) {
-      row = new Object[dimensionCount + msrCount];
-      fillDimensionData(convertedResult, queryDimensions, dimensionCount, row, rowIndex);
-
-      QueryMeasure msr;
-      for (int i = 0; i < queryModel.getQueryMeasures().size(); i++) {
-        msr = queryModel.getQueryMeasures().get(i);
-        row[msr.getQueryOrder()] = convertedResult[dimensionCount + i][rowIndex];
-      }
-      rows[rowIndex] = row;
-    }
-    LOGGER.info(
-        "###########################################------ Total Number of records" + rowSize);
-    BatchResult chunkResult = new BatchResult();
-    chunkResult.setRows(rows);
-    return chunkResult;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java b/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
deleted file mode 100644
index a6a8fbc..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/preparator/impl/RawQueryResultPreparatorImpl.java
+++ /dev/null
@@ -1,127 +0,0 @@
-package org.carbondata.query.carbon.result.preparator.impl;
-
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.encoder.Encoding;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
-import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.model.QuerySchemaInfo;
-import org.carbondata.query.carbon.result.BatchRawResult;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * It does not decode the dictionary.
- */
-public class RawQueryResultPreparatorImpl
-    extends AbstractQueryResultPreparator<List<ListBasedResultWrapper>, Object> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(RawQueryResultPreparatorImpl.class.getName());
-
-  private QuerySchemaInfo querySchemaInfo;
-
-  public RawQueryResultPreparatorImpl(QueryExecutorProperties executerProperties,
-      QueryModel queryModel) {
-    super(executerProperties, queryModel);
-    querySchemaInfo = new QuerySchemaInfo();
-    querySchemaInfo.setKeyGenerator(queryExecuterProperties.keyStructureInfo.getKeyGenerator());
-    querySchemaInfo.setMaskedByteIndexes(queryExecuterProperties.keyStructureInfo.getMaskedBytes());
-    querySchemaInfo.setQueryDimensions(queryModel.getQueryDimension()
-        .toArray(new QueryDimension[queryModel.getQueryDimension().size()]));
-    querySchemaInfo.setQueryMeasures(queryModel.getQueryMeasures()
-        .toArray(new QueryMeasure[queryModel.getQueryMeasures().size()]));
-    int msrSize = queryExecuterProperties.measureDataTypes.length;
-    int dimensionCount = queryModel.getQueryDimension().size();
-    int[] queryOrder = new int[dimensionCount + msrSize];
-    int[] queryReverseOrder = new int[dimensionCount + msrSize];
-    for (int i = 0; i < dimensionCount; i++) {
-      queryOrder[queryModel.getQueryDimension().get(i).getQueryOrder()] = i;
-      queryReverseOrder[i] = queryModel.getQueryDimension().get(i).getQueryOrder();
-    }
-    for (int i = 0; i < msrSize; i++) {
-      queryOrder[queryModel.getQueryMeasures().get(i).getQueryOrder()] = i + dimensionCount;
-      queryReverseOrder[i + dimensionCount] = queryModel.getQueryMeasures().get(i).getQueryOrder();
-    }
-    querySchemaInfo.setQueryOrder(queryOrder);
-    querySchemaInfo.setQueryReverseOrder(queryReverseOrder);
-  }
-
-  @Override public BatchResult prepareQueryResult(
-      Result<List<ListBasedResultWrapper>, Object> scannedResult) {
-    if ((null == scannedResult || scannedResult.size() < 1)) {
-      return new BatchRawResult();
-    }
-    QueryDimension[] queryDimensions = querySchemaInfo.getQueryDimensions();
-    int msrSize = queryExecuterProperties.measureDataTypes.length;
-    int dimSize = queryDimensions.length;
-    int[] order = querySchemaInfo.getQueryReverseOrder();
-    Object[][] resultData = new Object[scannedResult.size()][];
-    Object[] value;
-    Object[] row;
-    int counter = 0;
-    if (queryModel.isRawBytesDetailQuery()) {
-      while (scannedResult.hasNext()) {
-        value = scannedResult.getValue();
-        row = new Object[msrSize + 1];
-        row[0] = scannedResult.getKey();
-        if (value != null) {
-          assert (value.length == msrSize);
-          System.arraycopy(value, 0, row, 1, msrSize);
-        }
-        resultData[counter] = row;
-        counter++;
-      }
-    } else {
-      while (scannedResult.hasNext()) {
-        value = scannedResult.getValue();
-        row = new Object[msrSize + dimSize];
-        ByteArrayWrapper key = scannedResult.getKey();
-        if (key != null) {
-          long[] surrogateResult = querySchemaInfo.getKeyGenerator()
-              .getKeyArray(key.getDictionaryKey(), querySchemaInfo.getMaskedByteIndexes());
-          int noDictionaryColumnIndex = 0;
-          for (int i = 0; i < dimSize; i++) {
-            if (!queryDimensions[i].getDimension().hasEncoding(Encoding.DICTIONARY)) {
-              row[order[i]] = DataTypeUtil.getDataBasedOnDataType(
-                  new String(key.getNoDictionaryKeyByIndex(noDictionaryColumnIndex++)),
-                  queryDimensions[i].getDimension().getDataType());
-            } else if (queryDimensions[i].getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
-              DirectDictionaryGenerator directDictionaryGenerator =
-                  DirectDictionaryKeyGeneratorFactory.getDirectDictionaryGenerator(
-                      queryDimensions[i].getDimension().getDataType());
-              if (directDictionaryGenerator != null) {
-                row[order[i]] = directDictionaryGenerator.getValueFromSurrogate(
-                    (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()]);
-              }
-            } else {
-              row[order[i]] =
-                  (int) surrogateResult[queryDimensions[i].getDimension().getKeyOrdinal()];
-            }
-          }
-        }
-        for (int i = 0; i < msrSize; i++) {
-          row[order[i + queryDimensions.length]] = value[i];
-        }
-        resultData[counter] = row;
-        counter++;
-      }
-    }
-
-    LOGGER.info("###########################---- Total Number of records" + scannedResult.size());
-    BatchRawResult result = new BatchRawResult();
-    result.setRows(resultData);
-    return result;
-  }
-
-}
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java b/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java
deleted file mode 100644
index 34738c8..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/scanner/AbstractBlockletScanner.java
+++ /dev/null
@@ -1,62 +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.carbondata.query.carbon.scanner;
-
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Blocklet scanner class to process the block
- */
-public abstract class AbstractBlockletScanner implements BlockletScanner {
-
-  /**
-   * scanner result
-   */
-  protected AbstractScannedResult scannedResult;
-
-  /**
-   * block execution info
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  public AbstractBlockletScanner(BlockExecutionInfo tableBlockExecutionInfos) {
-    this.blockExecutionInfo = tableBlockExecutionInfos;
-  }
-
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws QueryExecutionException {
-    fillKeyValue(blocksChunkHolder);
-    return scannedResult;
-  }
-
-  protected void fillKeyValue(BlocksChunkHolder blocksChunkHolder) {
-    scannedResult.reset();
-    scannedResult.setMeasureChunks(blocksChunkHolder.getDataBlock()
-        .getMeasureChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedMeasureBlocksIndexes()));
-    scannedResult.setNumberOfRows(blocksChunkHolder.getDataBlock().nodeSize());
-
-    scannedResult.setDimensionChunks(blocksChunkHolder.getDataBlock()
-        .getDimensionChunks(blocksChunkHolder.getFileReader(),
-            blockExecutionInfo.getAllSelectedDimensionBlocksIndexes()));
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java b/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java
deleted file mode 100644
index 39b2568..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/scanner/BlockletScanner.java
+++ /dev/null
@@ -1,41 +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.carbondata.query.carbon.scanner;
-
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Interface for processing the block
- * Processing can be filter based processing or non filter based processing
- */
-public interface BlockletScanner {
-
-  /**
-   * Below method will used to process the block data and get the scanned result
-   *
-   * @param blocksChunkHolder block chunk which holds the block data
-   * @return scannerResult
-   * result after processing
-   * @throws QueryExecutionException
-   */
-  AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws QueryExecutionException;
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java
deleted file mode 100644
index 5225b41..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/FilterScanner.java
+++ /dev/null
@@ -1,174 +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.carbondata.query.carbon.scanner.impl;
-
-import java.util.BitSet;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.impl.FilterQueryScannedResult;
-import org.carbondata.query.carbon.scanner.AbstractBlockletScanner;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.executer.FilterExecuter;
-
-/**
- * Below class will be used for filter query processing
- * this class will be first apply the filter then it will read the block if
- * required and return the scanned result
- */
-public class FilterScanner extends AbstractBlockletScanner {
-
-  /**
-   * filter tree
-   */
-  private FilterExecuter filterExecuter;
-
-  /**
-   * this will be used to apply min max
-   * this will be useful for dimension column which is on the right side
-   * as node finder will always give tentative blocks, if column data stored individually
-   * and data is in sorted order then we can check whether filter is in the range of min max or not
-   * if it present then only we can apply filter on complete data.
-   * this will be very useful in case of sparse data when rows are
-   * repeating.
-   */
-  private boolean isMinMaxEnabled;
-
-  public FilterScanner(BlockExecutionInfo blockExecutionInfo) {
-    super(blockExecutionInfo);
-    scannedResult = new FilterQueryScannedResult(blockExecutionInfo);
-    // to check whether min max is enabled or not
-    String minMaxEnableValue = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_QUERY_MIN_MAX_ENABLED,
-            CarbonCommonConstants.MIN_MAX_DEFAULT_VALUE);
-    if (null != minMaxEnableValue) {
-      isMinMaxEnabled = Boolean.parseBoolean(minMaxEnableValue);
-    }
-    // get the filter tree
-    this.filterExecuter = blockExecutionInfo.getFilterExecuterTree();
-  }
-
-  /**
-   * Below method will be used to process the block
-   *
-   * @param blocksChunkHolder block chunk holder which holds the data
-   * @throws QueryExecutionException
-   * @throws FilterUnsupportedException
-   */
-  @Override public AbstractScannedResult scanBlocklet(BlocksChunkHolder blocksChunkHolder)
-      throws QueryExecutionException {
-    try {
-      fillScannedResult(blocksChunkHolder);
-    } catch (FilterUnsupportedException e) {
-      throw new QueryExecutionException(e.getMessage());
-    }
-    return scannedResult;
-  }
-
-  /**
-   * This method will process the data in below order
-   * 1. first apply min max on the filter tree and check whether any of the filter
-   * is fall on the range of min max, if not then return empty result
-   * 2. If filter falls on min max range then apply filter on actual
-   * data and get the filtered row index
-   * 3. if row index is empty then return the empty result
-   * 4. if row indexes is not empty then read only those blocks(measure or dimension)
-   * which was present in the query but not present in the filter, as while applying filter
-   * some of the blocks where already read and present in chunk holder so not need to
-   * read those blocks again, this is to avoid reading of same blocks which was already read
-   * 5. Set the blocks and filter indexes to result
-   *
-   * @param blocksChunkHolder
-   * @throws FilterUnsupportedException
-   */
-  private void fillScannedResult(BlocksChunkHolder blocksChunkHolder)
-      throws FilterUnsupportedException {
-
-    scannedResult.reset();
-    // apply min max
-    if (isMinMaxEnabled) {
-      BitSet bitSet = this.filterExecuter
-          .isScanRequired(blocksChunkHolder.getDataBlock().getColumnsMaxValue(),
-              blocksChunkHolder.getDataBlock().getColumnsMinValue());
-      if (bitSet.isEmpty()) {
-        scannedResult.setNumberOfRows(0);
-        scannedResult.setIndexes(new int[0]);
-        return;
-      }
-    }
-    // apply filter on actual data
-    BitSet bitSet = this.filterExecuter.applyFilter(blocksChunkHolder);
-    // if indexes is empty then return with empty result
-    if (bitSet.isEmpty()) {
-      scannedResult.setNumberOfRows(0);
-      scannedResult.setIndexes(new int[0]);
-      return;
-    }
-    // get the row indexes from bot set
-    int[] indexes = new int[bitSet.cardinality()];
-    int index = 0;
-    for (int i = bitSet.nextSetBit(0); i >= 0; i = bitSet.nextSetBit(i + 1)) {
-      indexes[index++] = i;
-    }
-
-    FileHolder fileReader = blocksChunkHolder.getFileReader();
-    int[] allSelectedDimensionBlocksIndexes =
-        blockExecutionInfo.getAllSelectedDimensionBlocksIndexes();
-    DimensionColumnDataChunk[] dimensionColumnDataChunk =
-        new DimensionColumnDataChunk[blockExecutionInfo.getTotalNumberDimensionBlock()];
-    // read dimension chunk blocks from file which is not present
-    for (int i = 0; i < allSelectedDimensionBlocksIndexes.length; i++) {
-      if (null == blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]]) {
-        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getDimensionChunk(fileReader, allSelectedDimensionBlocksIndexes[i]);
-      } else {
-        dimensionColumnDataChunk[allSelectedDimensionBlocksIndexes[i]] =
-            blocksChunkHolder.getDimensionDataChunk()[allSelectedDimensionBlocksIndexes[i]];
-      }
-    }
-    MeasureColumnDataChunk[] measureColumnDataChunk =
-        new MeasureColumnDataChunk[blockExecutionInfo.getTotalNumberOfMeasureBlock()];
-    int[] allSelectedMeasureBlocksIndexes = blockExecutionInfo.getAllSelectedMeasureBlocksIndexes();
-
-    // read the measure chunk blocks which is not present
-    for (int i = 0; i < allSelectedMeasureBlocksIndexes.length; i++) {
-
-      if (null == blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]]) {
-        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
-            blocksChunkHolder.getDataBlock()
-                .getMeasureChunk(fileReader, allSelectedMeasureBlocksIndexes[i]);
-      } else {
-        measureColumnDataChunk[allSelectedMeasureBlocksIndexes[i]] =
-            blocksChunkHolder.getMeasureDataChunk()[allSelectedMeasureBlocksIndexes[i]];
-      }
-    }
-    scannedResult.setDimensionChunks(dimensionColumnDataChunk);
-    scannedResult.setIndexes(indexes);
-    scannedResult.setMeasureChunks(measureColumnDataChunk);
-    scannedResult.setNumberOfRows(indexes.length);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.java b/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.java
deleted file mode 100644
index c42d8b8..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/scanner/impl/NonFilterScanner.java
+++ /dev/null
@@ -1,37 +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.carbondata.query.carbon.scanner.impl;
-
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.impl.NonFilterQueryScannedResult;
-import org.carbondata.query.carbon.scanner.AbstractBlockletScanner;
-
-/**
- * Non filter processor which will be used for non filter query
- * In case of non filter query we just need to read all the blocks requested in the
- * query and pass it to scanned result
- */
-public class NonFilterScanner extends AbstractBlockletScanner {
-
-  public NonFilterScanner(BlockExecutionInfo blockExecutionInfo) {
-    super(blockExecutionInfo);
-    // as its a non filter query creating a non filter query scanned result object
-    scannedResult = new NonFilterQueryScannedResult(blockExecutionInfo);
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java b/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
deleted file mode 100644
index 2a7c5ae..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/util/DataTypeUtil.java
+++ /dev/null
@@ -1,175 +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.carbondata.query.carbon.util;
-
-import java.math.BigDecimal;
-import java.text.ParseException;
-import java.text.SimpleDateFormat;
-import java.util.Date;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.util.CarbonProperties;
-
-import org.apache.spark.unsafe.types.UTF8String;
-
-/**
- * Utility for data type
- */
-public class DataTypeUtil {
-
-  /**
-   * LOGGER
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(DataTypeUtil.class.getName());
-
-  /**
-   * Below method will be used to convert the data passed to its actual data
-   * type
-   *
-   * @param data           data
-   * @param actualDataType actual data type
-   * @return actual data after conversion
-   */
-  public static Object getDataBasedOnDataType(String data, DataType actualDataType) {
-
-    if (null == data) {
-      return null;
-    }
-    try {
-      switch (actualDataType) {
-        case INT:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Integer.parseInt(data);
-        case DOUBLE:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Double.parseDouble(data);
-        case LONG:
-          if (data.isEmpty()) {
-            return null;
-          }
-          return Long.parseLong(data);
-        case TIMESTAMP:
-          if (data.isEmpty()) {
-            return null;
-          }
-          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
-          Date dateToStr = null;
-          try {
-            dateToStr = parser.parse(data);
-            return dateToStr.getTime() * 1000;
-          } catch (ParseException e) {
-            LOGGER.error("Cannot convert" + data + " to Time/Long type value" + e.getMessage());
-            return null;
-          }
-        case DECIMAL:
-          if (data.isEmpty()) {
-            return null;
-          }
-          java.math.BigDecimal javaDecVal = new java.math.BigDecimal(data);
-          scala.math.BigDecimal scalaDecVal = new scala.math.BigDecimal(javaDecVal);
-          org.apache.spark.sql.types.Decimal decConverter =
-              new org.apache.spark.sql.types.Decimal();
-          return decConverter.set(scalaDecVal);
-        default:
-          return UTF8String.fromString(data);
-      }
-    } catch (NumberFormatException ex) {
-      LOGGER.error("Problem while converting data type" + data);
-      return null;
-    }
-
-  }
-
-  public static Object getMeasureDataBasedOnDataType(Object data, DataType dataType) {
-
-    if (null == data) {
-      return null;
-    }
-    try {
-      switch (dataType) {
-        case DOUBLE:
-
-          return (Double) data;
-        case LONG:
-
-          return (Long) data;
-
-        case DECIMAL:
-
-          java.math.BigDecimal javaDecVal = new java.math.BigDecimal(data.toString());
-          scala.math.BigDecimal scalaDecVal = new scala.math.BigDecimal(javaDecVal);
-          org.apache.spark.sql.types.Decimal decConverter =
-              new org.apache.spark.sql.types.Decimal();
-          return decConverter.set(scalaDecVal);
-        default:
-
-          return data;
-      }
-    } catch (NumberFormatException ex) {
-      LOGGER.error("Problem while converting data type" + data);
-      return null;
-    }
-
-  }
-
-  public static int compareBasedOnDatatYpe(Object data1, Object data2, DataType dataType) {
-    switch (dataType) {
-      case INT:
-        return ((Integer) data1).compareTo((Integer) data2);
-      case LONG:
-      case TIMESTAMP:
-        return ((Long) data1).compareTo((Long) data2);
-      case DOUBLE:
-        return ((Double) data1).compareTo((Double) data2);
-      case DECIMAL:
-        return ((BigDecimal) data1).compareTo((BigDecimal) data2);
-      default:
-        return ((String) data1).compareTo((String) data2);
-    }
-  }
-
-  /**
-   * below method is to check whether data type is of numeric type or not
-   *
-   * @param dataType data type
-   * @return true if numeric data type
-   */
-  public boolean isNumericDatatype(DataType dataType) {
-    switch (dataType) {
-      case INT:
-      case LONG:
-      case DOUBLE:
-      case DECIMAL:
-        return true;
-      default:
-        return false;
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java
deleted file mode 100644
index 6050b97..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/wrappers/ByteArrayWrapper.java
+++ /dev/null
@@ -1,202 +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.carbondata.query.carbon.wrappers;
-
-import org.carbondata.core.util.ByteUtil.UnsafeComparer;
-
-/**
- * This class will store the dimension column data when query is executed
- * This can be used as a key for aggregation
- */
-public class ByteArrayWrapper implements Comparable<ByteArrayWrapper> {
-
-  /**
-   * to store key which is generated using
-   * key generator
-   */
-  protected byte[] dictionaryKey;
-
-  /**
-   * to store no dictionary column data
-   */
-  protected byte[][] complexTypesKeys;
-
-  /**
-   * to store no dictionary column data
-   */
-  protected byte[][] noDictionaryKeys;
-
-  public ByteArrayWrapper() {
-  }
-
-  /**
-   * @return the dictionaryKey
-   */
-  public byte[] getDictionaryKey() {
-    return dictionaryKey;
-  }
-
-  /**
-   * @param dictionaryKey the dictionaryKey to set
-   */
-  public void setDictionaryKey(byte[] dictionaryKey) {
-    this.dictionaryKey = dictionaryKey;
-  }
-
-  /**
-   * @param noDictionaryKeys the noDictionaryKeys to set
-   */
-  public void setNoDictionaryKeys(byte[][] noDictionaryKeys) {
-    this.noDictionaryKeys = noDictionaryKeys;
-  }
-
-  /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getNoDictionaryKeyByIndex(int index) {
-    return this.noDictionaryKeys[index];
-  }
-
-  /**
-   * to get the no dictionary column data
-   *
-   * @param index of the no dictionary key
-   * @return no dictionary key for the index
-   */
-  public byte[] getComplexTypeByIndex(int index) {
-    return this.complexTypesKeys[index];
-  }
-
-  /**
-   * to generate the hash code
-   */
-  @Override public int hashCode() {
-    // first generate the has code of the dictionary column
-    int len = dictionaryKey.length;
-    int result = 1;
-    for (int j = 0; j < len; j++) {
-      result = 31 * result + dictionaryKey[j];
-    }
-    // then no dictionary column
-    for (byte[] directSurrogateValue : noDictionaryKeys) {
-      for (int i = 0; i < directSurrogateValue.length; i++) {
-        result = 31 * result + directSurrogateValue[i];
-      }
-    }
-    // then for complex type
-    for (byte[] complexTypeKey : complexTypesKeys) {
-      for (int i = 0; i < complexTypeKey.length; i++) {
-        result = 31 * result + complexTypeKey[i];
-      }
-    }
-    return result;
-  }
-
-  /**
-   * to validate the two
-   *
-   * @param other object
-   */
-  @Override public boolean equals(Object other) {
-    if (null == other || !(other instanceof ByteArrayWrapper)) {
-      return false;
-    }
-    boolean result = false;
-    // Comparison will be as follows
-    // first compare the no dictionary column
-    // if it is not equal then return false
-    // if it is equal then compare the complex column
-    // if it is also equal then compare dictionary column
-    byte[][] noDictionaryKeysOther = ((ByteArrayWrapper) other).noDictionaryKeys;
-    if (noDictionaryKeysOther.length != noDictionaryKeys.length) {
-      return false;
-    } else {
-      for (int i = 0; i < noDictionaryKeys.length; i++) {
-        result = UnsafeComparer.INSTANCE.equals(noDictionaryKeys[i], noDictionaryKeysOther[i]);
-        if (!result) {
-          return false;
-        }
-      }
-    }
-
-    byte[][] complexTypesKeysOther = ((ByteArrayWrapper) other).complexTypesKeys;
-    if (complexTypesKeysOther.length != complexTypesKeys.length) {
-      return false;
-    } else {
-      for (int i = 0; i < complexTypesKeys.length; i++) {
-        result = UnsafeComparer.INSTANCE.equals(complexTypesKeys[i], complexTypesKeysOther[i]);
-        if (!result) {
-          return false;
-        }
-      }
-    }
-
-    return UnsafeComparer.INSTANCE.equals(dictionaryKey, ((ByteArrayWrapper) other).dictionaryKey);
-  }
-
-  /**
-   * Compare method for ByteArrayWrapper class this will used to compare Two
-   * ByteArrayWrapper data object, basically it will compare two byte array
-   *
-   * @param other ArrayWrapper Object
-   */
-  @Override public int compareTo(ByteArrayWrapper other) {
-    // compare will be as follows
-    //compare dictionary column
-    // then no dictionary column
-    // then complex type column data
-    int compareTo = UnsafeComparer.INSTANCE.compareTo(dictionaryKey, other.dictionaryKey);
-    if (compareTo == 0) {
-      for (int i = 0; i < noDictionaryKeys.length; i++) {
-        compareTo =
-            UnsafeComparer.INSTANCE.compareTo(noDictionaryKeys[i], other.noDictionaryKeys[i]);
-        if (compareTo != 0) {
-          return compareTo;
-        }
-      }
-    }
-    if (compareTo == 0) {
-      for (int i = 0; i < complexTypesKeys.length; i++) {
-        compareTo =
-            UnsafeComparer.INSTANCE.compareTo(complexTypesKeys[i], other.complexTypesKeys[i]);
-        if (compareTo != 0) {
-          return compareTo;
-        }
-      }
-    }
-    return compareTo;
-  }
-
-  /**
-   * @return the complexTypesKeys
-   */
-  public byte[][] getComplexTypesKeys() {
-    return complexTypesKeys;
-  }
-
-  /**
-   * @param complexTypesKeys the complexTypesKeys to set
-   */
-  public void setComplexTypesKeys(byte[][] complexTypesKeys) {
-    this.complexTypesKeys = complexTypesKeys;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.java b/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.java
deleted file mode 100644
index c54ce84..0000000
--- a/core/src/main/java/org/carbondata/query/carbonfilterinterface/ExpressionType.java
+++ /dev/null
@@ -1,44 +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.carbondata.query.carbonfilterinterface;
-
-public enum ExpressionType {
-
-  AND,
-  OR,
-  NOT,
-  EQUALS,
-  NOT_EQUALS,
-  LESSTHAN,
-  LESSTHAN_EQUALTO,
-  GREATERTHAN,
-  GREATERTHAN_EQUALTO,
-  ADD,
-  SUBSTRACT,
-  DIVIDE,
-  MULTIPLY,
-  IN,
-  LIST,
-  NOT_IN,
-  UNKNOWN,
-  LITERAL,
-  RANGE
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java b/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java
deleted file mode 100644
index b037c38..0000000
--- a/core/src/main/java/org/carbondata/query/carbonfilterinterface/FilterExecuterType.java
+++ /dev/null
@@ -1,28 +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.carbondata.query.carbonfilterinterface;
-
-import java.io.Serializable;
-
-public enum FilterExecuterType implements Serializable {
-
-  INCLUDE, EXCLUDE, OR, AND, RESTRUCTURE, ROWLEVEL, RANGE, ROWLEVEL_GREATERTHAN,
-  ROWLEVEL_GREATERTHAN_EQUALTO, ROWLEVEL_LESSTHAN_EQUALTO, ROWLEVEL_LESSTHAN
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.java b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.java
deleted file mode 100644
index 2133aca..0000000
--- a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowImpl.java
+++ /dev/null
@@ -1,44 +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.carbondata.query.carbonfilterinterface;
-
-public class RowImpl implements RowIntf {
-  private Object[] row;
-
-  public RowImpl() {
-    row = new Object[0];
-  }
-
-  @Override public Object getVal(int index) {
-    return row[index];
-  }
-
-  @Override public Object[] getValues() {
-    return row;
-  }
-
-  @Override public void setValues(final Object[] row) {
-    this.row = row;
-  }
-
-  @Override public int size() {
-    return this.row.length;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java b/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java
deleted file mode 100644
index 1984eed..0000000
--- a/core/src/main/java/org/carbondata/query/carbonfilterinterface/RowIntf.java
+++ /dev/null
@@ -1,31 +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.carbondata.query.carbonfilterinterface;
-
-public interface RowIntf {
-  Object getVal(int index);
-
-  Object[] getValues();
-
-  void setValues(Object[] setValues);
-
-  int size();
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java b/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java
deleted file mode 100644
index bfcc0f8..0000000
--- a/core/src/main/java/org/carbondata/query/complex/querytypes/GenericQueryType.java
+++ /dev/null
@@ -1,73 +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.carbondata.query.complex.querytypes;
-
-import java.io.DataOutputStream;
-import java.io.IOException;
-import java.nio.ByteBuffer;
-import java.util.List;
-
-import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
-import org.carbondata.query.carbon.processor.BlocksChunkHolder;
-
-import org.apache.spark.sql.types.DataType;
-
-public interface GenericQueryType {
-
-  String getName();
-
-  void setName(String name);
-
-  String getParentname();
-
-  void setParentname(String parentname);
-
-  int getBlockIndex();
-
-  void setBlockIndex(int blockIndex);
-
-  void addChildren(GenericQueryType children);
-
-  void getAllPrimitiveChildren(List<GenericQueryType> primitiveChild);
-
-  int getSurrogateIndex();
-
-  void setSurrogateIndex(int surrIndex);
-
-  int getColsCount();
-
-  void setKeySize(int[] keyBlockSize);
-
-  int getKeyOrdinalForQuery();
-
-  void setKeyOrdinalForQuery(int keyOrdinalForQuery);
-
-  void parseBlocksAndReturnComplexColumnByteArray(
-      ColumnarKeyStoreDataHolder[] columnarKeyStoreDataHolder, int rowNumber,
-      DataOutputStream dataOutputStream) throws IOException;
-
-  DataType getSchemaType();
-
-  void parseAndGetResultBytes(ByteBuffer complexData, DataOutputStream dataOutput)
-      throws IOException;
-
-  void fillRequiredBlockData(BlocksChunkHolder blockChunkHolder);
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java b/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java
deleted file mode 100644
index dd2a656..0000000
--- a/core/src/main/java/org/carbondata/query/evaluators/DimColumnExecuterFilterInfo.java
+++ /dev/null
@@ -1,32 +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.carbondata.query.evaluators;
-
-public class DimColumnExecuterFilterInfo {
-
-  byte[][] filterKeys;
-
-  public void setFilterKeys(byte[][] filterKeys) {
-    this.filterKeys = filterKeys;
-  }
-
-  public byte[][] getFilterKeys() {
-    return filterKeys;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.java b/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.java
deleted file mode 100644
index 174336b..0000000
--- a/core/src/main/java/org/carbondata/query/evaluators/FilterProcessorPlaceHolder.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.carbondata.query.evaluators;
-
-public interface FilterProcessorPlaceHolder {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java b/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java
deleted file mode 100644
index c85d991..0000000
--- a/core/src/main/java/org/carbondata/query/expression/BinaryExpression.java
+++ /dev/null
@@ -1,59 +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.carbondata.query.expression;
-
-public abstract class BinaryExpression extends Expression {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-  /**
-   *
-   */
-
-  protected Expression left;
-  protected Expression right;
-  protected boolean isRangeExpression;
-
-  public BinaryExpression(Expression left, Expression right) {
-    this.left = left;
-    this.right = right;
-    children.add(left);
-    children.add(right);
-  }
-
-  public Expression getLeft() {
-    return left;
-  }
-
-  public Expression getRight() {
-    return right;
-  }
-
-  public boolean isRangeExpression() {
-    return isRangeExpression;
-  }
-
-  public void setRangeExpression(boolean isRangeExpression) {
-    this.isRangeExpression = isRangeExpression;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java b/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
deleted file mode 100644
index aa0de43..0000000
--- a/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
+++ /dev/null
@@ -1,114 +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.carbondata.query.expression;
-
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-
-public class ColumnExpression extends LeafExpression {
-  /**
-   *
-   */
-  private static final long serialVersionUID = 1L;
-
-  private String columnName;
-
-  private boolean isDimension;
-
-  private int colIndex = -1;
-
-  private DataType dataType;
-
-  private CarbonDimension dimension;
-
-  private CarbonColumn carbonColumn;
-
-  public ColumnExpression(String columnName, DataType dataType) {
-    this.columnName = columnName;
-    this.dataType = dataType;
-
-  }
-
-  public CarbonDimension getDimension() {
-    return dimension;
-  }
-
-  public void setDimension(CarbonDimension dimension) {
-    this.dimension = dimension;
-  }
-
-  public String getColumnName() {
-    return columnName;
-  }
-
-  public void setColumnName(String columnName) {
-    this.columnName = columnName;
-  }
-
-  public boolean isDimension() {
-    return isDimension;
-  }
-
-  public void setDimension(boolean isDimension) {
-    this.isDimension = isDimension;
-  }
-
-  public int getColIndex() {
-    return colIndex;
-  }
-
-  public void setColIndex(int colIndex) {
-    this.colIndex = colIndex;
-  }
-
-  public DataType getDataType() {
-    return dataType;
-  }
-
-  public void setDataType(DataType dataType) {
-    this.dataType = dataType;
-  }
-
-  @Override public ExpressionResult evaluate(RowIntf value) {
-    ExpressionResult expressionResult = new ExpressionResult(dataType, value.getVal(colIndex));
-    return expressionResult;
-  }
-
-  @Override public ExpressionType getFilterExpressionType() {
-    // TODO Auto-generated method stub
-    return null;
-  }
-
-  @Override public String getString() {
-    // TODO Auto-generated method stub
-    return "ColumnExpression(" + columnName + ')';
-  }
-
-  public CarbonColumn getCarbonColumn() {
-    return carbonColumn;
-  }
-
-  public void setCarbonColumn(CarbonColumn carbonColumn) {
-    this.carbonColumn = carbonColumn;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/DataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/DataType.java b/core/src/main/java/org/carbondata/query/expression/DataType.java
deleted file mode 100644
index fb2b277..0000000
--- a/core/src/main/java/org/carbondata/query/expression/DataType.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.carbondata.query.expression;
-
-public enum DataType {
-  StringType(0), DateType(1), TimestampType(2), BooleanType(1), IntegerType(3), FloatType(
-      4), LongType(5), DoubleType(6), NullType(7), DecimalType(8), ArrayType(9), StructType(10);
-  private int presedenceOrder;
-
-  private DataType(int value) {
-    this.presedenceOrder = value;
-  }
-
-  public int getPresedenceOrder() {
-    return presedenceOrder;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/expression/Expression.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/expression/Expression.java b/core/src/main/java/org/carbondata/query/expression/Expression.java
deleted file mode 100644
index c033020..0000000
--- a/core/src/main/java/org/carbondata/query/expression/Expression.java
+++ /dev/null
@@ -1,51 +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.carbondata.query.expression;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbonfilterinterface.ExpressionType;
-import org.carbondata.query.carbonfilterinterface.RowIntf;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-
-public abstract class Expression implements Serializable {
-
-  /**
-   *
-   */
-  private static final long serialVersionUID = -7568676723039530713L;
-  protected List<Expression> children =
-      new ArrayList<Expression>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-
-  public abstract ExpressionResult evaluate(RowIntf value) throws FilterUnsupportedException;
-
-  public abstract ExpressionType getFilterExpressionType();
-
-  public List<Expression> getChildren() {
-    return children;
-  }
-
-  public abstract String getString();
-
-  // public abstract void  accept(ExpressionVisitor visitor);
-}


[25/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java b/core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
new file mode 100644
index 0000000..8f301ca
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
@@ -0,0 +1,175 @@
+/*
+ * 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.carbondata.scan.util;
+
+import java.math.BigDecimal;
+import java.text.ParseException;
+import java.text.SimpleDateFormat;
+import java.util.Date;
+
+import org.carbondata.common.logging.LogService;
+import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonProperties;
+
+import org.apache.spark.unsafe.types.UTF8String;
+
+/**
+ * Utility for data type
+ */
+public class DataTypeUtil {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(DataTypeUtil.class.getName());
+
+  /**
+   * Below method will be used to convert the data passed to its actual data
+   * type
+   *
+   * @param data           data
+   * @param actualDataType actual data type
+   * @return actual data after conversion
+   */
+  public static Object getDataBasedOnDataType(String data, DataType actualDataType) {
+
+    if (null == data) {
+      return null;
+    }
+    try {
+      switch (actualDataType) {
+        case INT:
+          if (data.isEmpty()) {
+            return null;
+          }
+          return Integer.parseInt(data);
+        case DOUBLE:
+          if (data.isEmpty()) {
+            return null;
+          }
+          return Double.parseDouble(data);
+        case LONG:
+          if (data.isEmpty()) {
+            return null;
+          }
+          return Long.parseLong(data);
+        case TIMESTAMP:
+          if (data.isEmpty()) {
+            return null;
+          }
+          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
+                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+          Date dateToStr = null;
+          try {
+            dateToStr = parser.parse(data);
+            return dateToStr.getTime() * 1000;
+          } catch (ParseException e) {
+            LOGGER.error("Cannot convert" + data + " to Time/Long type value" + e.getMessage());
+            return null;
+          }
+        case DECIMAL:
+          if (data.isEmpty()) {
+            return null;
+          }
+          java.math.BigDecimal javaDecVal = new java.math.BigDecimal(data);
+          scala.math.BigDecimal scalaDecVal = new scala.math.BigDecimal(javaDecVal);
+          org.apache.spark.sql.types.Decimal decConverter =
+              new org.apache.spark.sql.types.Decimal();
+          return decConverter.set(scalaDecVal);
+        default:
+          return UTF8String.fromString(data);
+      }
+    } catch (NumberFormatException ex) {
+      LOGGER.error("Problem while converting data type" + data);
+      return null;
+    }
+
+  }
+
+  public static Object getMeasureDataBasedOnDataType(Object data, DataType dataType) {
+
+    if (null == data) {
+      return null;
+    }
+    try {
+      switch (dataType) {
+        case DOUBLE:
+
+          return (Double) data;
+        case LONG:
+
+          return (Long) data;
+
+        case DECIMAL:
+
+          java.math.BigDecimal javaDecVal = new java.math.BigDecimal(data.toString());
+          scala.math.BigDecimal scalaDecVal = new scala.math.BigDecimal(javaDecVal);
+          org.apache.spark.sql.types.Decimal decConverter =
+              new org.apache.spark.sql.types.Decimal();
+          return decConverter.set(scalaDecVal);
+        default:
+
+          return data;
+      }
+    } catch (NumberFormatException ex) {
+      LOGGER.error("Problem while converting data type" + data);
+      return null;
+    }
+
+  }
+
+  public static int compareBasedOnDatatYpe(Object data1, Object data2, DataType dataType) {
+    switch (dataType) {
+      case INT:
+        return ((Integer) data1).compareTo((Integer) data2);
+      case LONG:
+      case TIMESTAMP:
+        return ((Long) data1).compareTo((Long) data2);
+      case DOUBLE:
+        return ((Double) data1).compareTo((Double) data2);
+      case DECIMAL:
+        return ((BigDecimal) data1).compareTo((BigDecimal) data2);
+      default:
+        return ((String) data1).compareTo((String) data2);
+    }
+  }
+
+  /**
+   * below method is to check whether data type is of numeric type or not
+   *
+   * @param dataType data type
+   * @return true if numeric data type
+   */
+  public boolean isNumericDatatype(DataType dataType) {
+    switch (dataType) {
+      case INT:
+      case LONG:
+      case DOUBLE:
+      case DECIMAL:
+        return true;
+      default:
+        return false;
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/scan/wrappers/ByteArrayWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/wrappers/ByteArrayWrapper.java b/core/src/main/java/org/carbondata/scan/wrappers/ByteArrayWrapper.java
new file mode 100644
index 0000000..d36ef7f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/wrappers/ByteArrayWrapper.java
@@ -0,0 +1,202 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.scan.wrappers;
+
+import org.carbondata.core.util.ByteUtil.UnsafeComparer;
+
+/**
+ * This class will store the dimension column data when query is executed
+ * This can be used as a key for aggregation
+ */
+public class ByteArrayWrapper implements Comparable<ByteArrayWrapper> {
+
+  /**
+   * to store key which is generated using
+   * key generator
+   */
+  protected byte[] dictionaryKey;
+
+  /**
+   * to store no dictionary column data
+   */
+  protected byte[][] complexTypesKeys;
+
+  /**
+   * to store no dictionary column data
+   */
+  protected byte[][] noDictionaryKeys;
+
+  public ByteArrayWrapper() {
+  }
+
+  /**
+   * @return the dictionaryKey
+   */
+  public byte[] getDictionaryKey() {
+    return dictionaryKey;
+  }
+
+  /**
+   * @param dictionaryKey the dictionaryKey to set
+   */
+  public void setDictionaryKey(byte[] dictionaryKey) {
+    this.dictionaryKey = dictionaryKey;
+  }
+
+  /**
+   * @param noDictionaryKeys the noDictionaryKeys to set
+   */
+  public void setNoDictionaryKeys(byte[][] noDictionaryKeys) {
+    this.noDictionaryKeys = noDictionaryKeys;
+  }
+
+  /**
+   * to get the no dictionary column data
+   *
+   * @param index of the no dictionary key
+   * @return no dictionary key for the index
+   */
+  public byte[] getNoDictionaryKeyByIndex(int index) {
+    return this.noDictionaryKeys[index];
+  }
+
+  /**
+   * to get the no dictionary column data
+   *
+   * @param index of the no dictionary key
+   * @return no dictionary key for the index
+   */
+  public byte[] getComplexTypeByIndex(int index) {
+    return this.complexTypesKeys[index];
+  }
+
+  /**
+   * to generate the hash code
+   */
+  @Override public int hashCode() {
+    // first generate the has code of the dictionary column
+    int len = dictionaryKey.length;
+    int result = 1;
+    for (int j = 0; j < len; j++) {
+      result = 31 * result + dictionaryKey[j];
+    }
+    // then no dictionary column
+    for (byte[] directSurrogateValue : noDictionaryKeys) {
+      for (int i = 0; i < directSurrogateValue.length; i++) {
+        result = 31 * result + directSurrogateValue[i];
+      }
+    }
+    // then for complex type
+    for (byte[] complexTypeKey : complexTypesKeys) {
+      for (int i = 0; i < complexTypeKey.length; i++) {
+        result = 31 * result + complexTypeKey[i];
+      }
+    }
+    return result;
+  }
+
+  /**
+   * to validate the two
+   *
+   * @param other object
+   */
+  @Override public boolean equals(Object other) {
+    if (null == other || !(other instanceof ByteArrayWrapper)) {
+      return false;
+    }
+    boolean result = false;
+    // Comparison will be as follows
+    // first compare the no dictionary column
+    // if it is not equal then return false
+    // if it is equal then compare the complex column
+    // if it is also equal then compare dictionary column
+    byte[][] noDictionaryKeysOther = ((ByteArrayWrapper) other).noDictionaryKeys;
+    if (noDictionaryKeysOther.length != noDictionaryKeys.length) {
+      return false;
+    } else {
+      for (int i = 0; i < noDictionaryKeys.length; i++) {
+        result = UnsafeComparer.INSTANCE.equals(noDictionaryKeys[i], noDictionaryKeysOther[i]);
+        if (!result) {
+          return false;
+        }
+      }
+    }
+
+    byte[][] complexTypesKeysOther = ((ByteArrayWrapper) other).complexTypesKeys;
+    if (complexTypesKeysOther.length != complexTypesKeys.length) {
+      return false;
+    } else {
+      for (int i = 0; i < complexTypesKeys.length; i++) {
+        result = UnsafeComparer.INSTANCE.equals(complexTypesKeys[i], complexTypesKeysOther[i]);
+        if (!result) {
+          return false;
+        }
+      }
+    }
+
+    return UnsafeComparer.INSTANCE.equals(dictionaryKey, ((ByteArrayWrapper) other).dictionaryKey);
+  }
+
+  /**
+   * Compare method for ByteArrayWrapper class this will used to compare Two
+   * ByteArrayWrapper data object, basically it will compare two byte array
+   *
+   * @param other ArrayWrapper Object
+   */
+  @Override public int compareTo(ByteArrayWrapper other) {
+    // compare will be as follows
+    //compare dictionary column
+    // then no dictionary column
+    // then complex type column data
+    int compareTo = UnsafeComparer.INSTANCE.compareTo(dictionaryKey, other.dictionaryKey);
+    if (compareTo == 0) {
+      for (int i = 0; i < noDictionaryKeys.length; i++) {
+        compareTo =
+            UnsafeComparer.INSTANCE.compareTo(noDictionaryKeys[i], other.noDictionaryKeys[i]);
+        if (compareTo != 0) {
+          return compareTo;
+        }
+      }
+    }
+    if (compareTo == 0) {
+      for (int i = 0; i < complexTypesKeys.length; i++) {
+        compareTo =
+            UnsafeComparer.INSTANCE.compareTo(complexTypesKeys[i], other.complexTypesKeys[i]);
+        if (compareTo != 0) {
+          return compareTo;
+        }
+      }
+    }
+    return compareTo;
+  }
+
+  /**
+   * @return the complexTypesKeys
+   */
+  public byte[][] getComplexTypesKeys() {
+    return complexTypesKeys;
+  }
+
+  /**
+   * @param complexTypesKeys the complexTypesKeys to set
+   */
+  public void setComplexTypesKeys(byte[][] complexTypesKeys) {
+    this.complexTypesKeys = complexTypesKeys;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/test/java/org/carbondata/query/QueryExecutor_UT.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/query/QueryExecutor_UT.java b/core/src/test/java/org/carbondata/query/QueryExecutor_UT.java
deleted file mode 100644
index a273b33..0000000
--- a/core/src/test/java/org/carbondata/query/QueryExecutor_UT.java
+++ /dev/null
@@ -1,35 +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.
- */
-
-/**
- * Copyright Notice
- * =====================================
- * This file contains proprietary information of
- * Huawei Technologies India Pvt Ltd.
- * Copying or reproduction without prior written approval is prohibited.
- * Copyright (c) 2012
- * =====================================
- */
-package org.carbondata.query;
-
-import junit.framework.TestCase;
-
-public class QueryExecutor_UT extends TestCase {
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/test/java/org/carbondata/query/carbon/executor/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/query/carbon/executor/util/QueryUtilTest.java b/core/src/test/java/org/carbondata/query/carbon/executor/util/QueryUtilTest.java
deleted file mode 100644
index d765013..0000000
--- a/core/src/test/java/org/carbondata/query/carbon/executor/util/QueryUtilTest.java
+++ /dev/null
@@ -1,133 +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.carbondata.query.carbon.executor.util;
-
-import java.util.ArrayList;
-import java.util.Arrays;
-import java.util.List;
-
-import junit.framework.TestCase;
-
-import org.carbondata.core.carbon.datastore.block.SegmentProperties;
-import org.carbondata.core.carbon.datastore.block.SegmentPropertiesTestUtil;
-import org.carbondata.core.keygenerator.KeyGenException;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.junit.AfterClass;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class QueryUtilTest extends TestCase {
-
-  private SegmentProperties segmentProperties;
-
-  @BeforeClass public void setUp() {
-    segmentProperties = SegmentPropertiesTestUtil.getSegmentProperties();
-  }
-
-  @Test public void testGetMaskedByteRangeGivingProperMaksedByteRange() {
-	  
-	QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-	dimension.setDimension(segmentProperties.getDimensions().get(0));  
-    int[] maskedByteRange = QueryUtil
-        .getMaskedByteRange(Arrays.asList(dimension),
-            segmentProperties.getDimensionKeyGenerator());
-    int[] expectedMaskedByteRange = { 0 };
-    for (int i = 0; i < maskedByteRange.length; i++) {
-      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
-    }
-  }
-
-  @Test public void testGetMaskedByteRangeGivingProperMaksedByteRangeOnlyForDictionaryKey() {
-    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
-    for (int i = 0; i < 2; i++) {
-      QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-      dimension.setDimension(segmentProperties.getDimensions().get(i));
-      dimensions.add(dimension);
-    }
-    int[] maskedByteRange =
-        QueryUtil.getMaskedByteRange(dimensions, segmentProperties.getDimensionKeyGenerator());
-    int[] expectedMaskedByteRange = { 0 };
-    for (int i = 0; i < maskedByteRange.length; i++) {
-      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
-    }
-  }
-
-  @Test public void testGetMaskedByteRangeBasedOrdinalGivingProperMaskedByte() {
-    List<Integer> dimensionOrdinal = new ArrayList<Integer>();
-    dimensionOrdinal.add(0);
-    int[] maskedByteRange = QueryUtil.getMaskedByteRangeBasedOrdinal(dimensionOrdinal,
-        segmentProperties.getDimensionKeyGenerator());
-    int[] expectedMaskedByteRange = { 0 };
-    for (int i = 0; i < maskedByteRange.length; i++) {
-      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
-    }
-  }
-
-  @Test public void testGetMaxKeyBasedOnDimensions() {
-	  List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
-	    for (int i = 0; i < 2; i++) {
-	      QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
-	      dimension.setDimension(segmentProperties.getDimensions().get(i));
-	      dimensions.add(dimension);
-	    }
-    byte[] maxKeyBasedOnDimensions = null;
-    try {
-      maxKeyBasedOnDimensions = QueryUtil
-          .getMaxKeyBasedOnDimensions(dimensions, segmentProperties.getDimensionKeyGenerator());
-    } catch (KeyGenException e) {
-      assertTrue(false);
-    }
-    byte[] expectedMaxKeyBasedOnDimensions = { -1, 0, 0, 0, 0, 0 };
-    for (int i = 0; i < expectedMaxKeyBasedOnDimensions.length; i++) {
-      if (expectedMaxKeyBasedOnDimensions[i] != maxKeyBasedOnDimensions[i]) {
-        assertTrue(false);
-      }
-    }
-    long[] expectedKeyArray = { 255, 0, 0, 0, 0, 0 };
-    long[] keyArray =
-        segmentProperties.getDimensionKeyGenerator().getKeyArray(maxKeyBasedOnDimensions);
-    for (int i = 0; i < keyArray.length; i++) {
-      if (expectedKeyArray[i] != keyArray[i]) {
-        assertTrue(false);
-      }
-    }
-  }
-
-  @Test public void testGetMaksedByte() {
-	  QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
-		dimension.setDimension(segmentProperties.getDimensions().get(0)); 
-		dimension.setDimension(segmentProperties.getDimensions().get(0));
-    int[] maskedByteRange = QueryUtil
-        .getMaskedByteRange(Arrays.asList(dimension),
-            segmentProperties.getDimensionKeyGenerator());
-    int[] maskedByte = QueryUtil
-        .getMaskedByte(segmentProperties.getDimensionKeyGenerator().getDimCount(), maskedByteRange);
-    int[] expectedMaskedByte = { 0, -1, -1, -1, -1, -1 };
-
-    for (int i = 0; i < expectedMaskedByte.length; i++) {
-      if (expectedMaskedByte[i] != maskedByte[i]) {
-        assertTrue(false);
-      }
-    }
-  }
-
-  @AfterClass public void tearDown() {
-    segmentProperties = null;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/test/java/org/carbondata/scan/QueryExecutor_UT.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/scan/QueryExecutor_UT.java b/core/src/test/java/org/carbondata/scan/QueryExecutor_UT.java
new file mode 100644
index 0000000..a04e9e9
--- /dev/null
+++ b/core/src/test/java/org/carbondata/scan/QueryExecutor_UT.java
@@ -0,0 +1,35 @@
+/*
+ * 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.
+ */
+
+/**
+ * Copyright Notice
+ * =====================================
+ * This file contains proprietary information of
+ * Huawei Technologies India Pvt Ltd.
+ * Copying or reproduction without prior written approval is prohibited.
+ * Copyright (c) 2012
+ * =====================================
+ */
+package org.carbondata.scan;
+
+import junit.framework.TestCase;
+
+public class QueryExecutor_UT extends TestCase {
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/test/java/org/carbondata/scan/executor/util/QueryUtilTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/scan/executor/util/QueryUtilTest.java b/core/src/test/java/org/carbondata/scan/executor/util/QueryUtilTest.java
new file mode 100644
index 0000000..0671c61
--- /dev/null
+++ b/core/src/test/java/org/carbondata/scan/executor/util/QueryUtilTest.java
@@ -0,0 +1,133 @@
+/*
+ * 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.carbondata.scan.executor.util;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.List;
+
+import junit.framework.TestCase;
+
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.datastore.block.SegmentPropertiesTestUtil;
+import org.carbondata.core.keygenerator.KeyGenException;
+import org.carbondata.scan.model.QueryDimension;
+import org.junit.AfterClass;
+import org.junit.BeforeClass;
+import org.junit.Test;
+
+public class QueryUtilTest extends TestCase {
+
+  private SegmentProperties segmentProperties;
+
+  @BeforeClass public void setUp() {
+    segmentProperties = SegmentPropertiesTestUtil.getSegmentProperties();
+  }
+
+  @Test public void testGetMaskedByteRangeGivingProperMaksedByteRange() {
+	  
+	QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
+	dimension.setDimension(segmentProperties.getDimensions().get(0));  
+    int[] maskedByteRange = QueryUtil
+        .getMaskedByteRange(Arrays.asList(dimension),
+            segmentProperties.getDimensionKeyGenerator());
+    int[] expectedMaskedByteRange = { 0 };
+    for (int i = 0; i < maskedByteRange.length; i++) {
+      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
+    }
+  }
+
+  @Test public void testGetMaskedByteRangeGivingProperMaksedByteRangeOnlyForDictionaryKey() {
+    List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+    for (int i = 0; i < 2; i++) {
+      QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
+      dimension.setDimension(segmentProperties.getDimensions().get(i));
+      dimensions.add(dimension);
+    }
+    int[] maskedByteRange =
+        QueryUtil.getMaskedByteRange(dimensions, segmentProperties.getDimensionKeyGenerator());
+    int[] expectedMaskedByteRange = { 0 };
+    for (int i = 0; i < maskedByteRange.length; i++) {
+      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
+    }
+  }
+
+  @Test public void testGetMaskedByteRangeBasedOrdinalGivingProperMaskedByte() {
+    List<Integer> dimensionOrdinal = new ArrayList<Integer>();
+    dimensionOrdinal.add(0);
+    int[] maskedByteRange = QueryUtil.getMaskedByteRangeBasedOrdinal(dimensionOrdinal,
+        segmentProperties.getDimensionKeyGenerator());
+    int[] expectedMaskedByteRange = { 0 };
+    for (int i = 0; i < maskedByteRange.length; i++) {
+      assertEquals(expectedMaskedByteRange[i], maskedByteRange[i]);
+    }
+  }
+
+  @Test public void testGetMaxKeyBasedOnDimensions() {
+	  List<QueryDimension> dimensions = new ArrayList<QueryDimension>();
+	    for (int i = 0; i < 2; i++) {
+	      QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(i).getColName());
+	      dimension.setDimension(segmentProperties.getDimensions().get(i));
+	      dimensions.add(dimension);
+	    }
+    byte[] maxKeyBasedOnDimensions = null;
+    try {
+      maxKeyBasedOnDimensions = QueryUtil
+          .getMaxKeyBasedOnDimensions(dimensions, segmentProperties.getDimensionKeyGenerator());
+    } catch (KeyGenException e) {
+      assertTrue(false);
+    }
+    byte[] expectedMaxKeyBasedOnDimensions = { -1, 0, 0, 0, 0, 0 };
+    for (int i = 0; i < expectedMaxKeyBasedOnDimensions.length; i++) {
+      if (expectedMaxKeyBasedOnDimensions[i] != maxKeyBasedOnDimensions[i]) {
+        assertTrue(false);
+      }
+    }
+    long[] expectedKeyArray = { 255, 0, 0, 0, 0, 0 };
+    long[] keyArray =
+        segmentProperties.getDimensionKeyGenerator().getKeyArray(maxKeyBasedOnDimensions);
+    for (int i = 0; i < keyArray.length; i++) {
+      if (expectedKeyArray[i] != keyArray[i]) {
+        assertTrue(false);
+      }
+    }
+  }
+
+  @Test public void testGetMaksedByte() {
+	  QueryDimension dimension = new QueryDimension(segmentProperties.getDimensions().get(0).getColName());
+		dimension.setDimension(segmentProperties.getDimensions().get(0)); 
+		dimension.setDimension(segmentProperties.getDimensions().get(0));
+    int[] maskedByteRange = QueryUtil
+        .getMaskedByteRange(Arrays.asList(dimension),
+            segmentProperties.getDimensionKeyGenerator());
+    int[] maskedByte = QueryUtil
+        .getMaskedByte(segmentProperties.getDimensionKeyGenerator().getDimCount(), maskedByteRange);
+    int[] expectedMaskedByte = { 0, -1, -1, -1, -1, -1 };
+
+    for (int i = 0; i < expectedMaskedByte.length; i++) {
+      if (expectedMaskedByte[i] != maskedByte[i]) {
+        assertTrue(false);
+      }
+    }
+  }
+
+  @AfterClass public void tearDown() {
+    segmentProperties = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/dev/findbugs-exclude.xml
----------------------------------------------------------------------
diff --git a/dev/findbugs-exclude.xml b/dev/findbugs-exclude.xml
index 3c0880c..4e1ca05 100644
--- a/dev/findbugs-exclude.xml
+++ b/dev/findbugs-exclude.xml
@@ -34,12 +34,12 @@
   </Match>
 
   <Match>
-    <Class name="org.carbondata.query.aggregator.impl.BitSet"/>
+    <Class name="org.carbondata.scan.aggregator.impl.BitSet"/>
     <Bug pattern="SE_TRANSIENT_FIELD_NOT_RESTORED"/>
   </Match>
 
   <Match>
-    <Class name="org.carbondata.query.expression.ExpressionResult"/>
+    <Class name="org.carbondata.scan.expression.ExpressionResult"/>
     <Or>
       <Method name="getBoolean"/>
     </Or>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/hadoop/src/main/java/org/carbondata/hadoop/CarbonInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/CarbonInputFormat.java b/hadoop/src/main/java/org/carbondata/hadoop/CarbonInputFormat.java
index 7f794e9..e9ec001 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/CarbonInputFormat.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/CarbonInputFormat.java
@@ -56,14 +56,14 @@ import org.carbondata.hadoop.util.CarbonInputFormatUtil;
 import org.carbondata.hadoop.util.ObjectSerializationUtil;
 import org.carbondata.hadoop.util.SchemaReader;
 import org.carbondata.lcm.status.SegmentStatusManager;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.model.CarbonQueryPlan;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.exception.FilterUnsupportedException;
-import org.carbondata.query.filter.resolver.FilterResolverIntf;
-import org.carbondata.query.filters.FilterExpressionProcessor;
-import org.carbondata.query.filters.measurefilter.util.FilterUtil;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.exception.FilterUnsupportedException;
+import org.carbondata.scan.filter.FilterExpressionProcessor;
+import org.carbondata.scan.filter.FilterUtil;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+import org.carbondata.scan.model.CarbonQueryPlan;
+import org.carbondata.scan.model.QueryModel;
 
 import static org.carbondata.core.constants.CarbonCommonConstants.INVALID_SEGMENT_ID;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
index 3d54d96..774ad14 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/CarbonRecordReader.java
@@ -10,11 +10,11 @@ import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.hadoop.readsupport.CarbonReadSupport;
-import org.carbondata.query.carbon.executor.QueryExecutorFactory;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.iterator.ChunkRowIterator;
+import org.carbondata.scan.executor.QueryExecutorFactory;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.result.BatchResult;
+import org.carbondata.scan.result.iterator.ChunkRowIterator;
 
 import org.apache.hadoop.mapreduce.InputSplit;
 import org.apache.hadoop.mapreduce.RecordReader;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java b/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java
index 7384636..279c8ab 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/util/CarbonInputFormatUtil.java
@@ -25,13 +25,13 @@ import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.query.carbon.model.CarbonQueryPlan;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.filter.resolver.FilterResolverIntf;
-import org.carbondata.query.filters.FilterExpressionProcessor;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.filter.FilterExpressionProcessor;
+import org.carbondata.scan.filter.resolver.FilterResolverIntf;
+import org.carbondata.scan.model.CarbonQueryPlan;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.model.QueryModel;
 
 /**
  * Utility class

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputFormat_FT.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputFormat_FT.java b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputFormat_FT.java
index 1cf4a87..bf5624b 100644
--- a/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputFormat_FT.java
+++ b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputFormat_FT.java
@@ -19,17 +19,16 @@
 
 package org.carbondata.hadoop.ft;
 
-import java.util.Arrays;
 import java.util.List;
 import java.util.UUID;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
 import org.carbondata.hadoop.CarbonInputFormat;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.LiteralExpression;
-import org.carbondata.query.expression.conditional.EqualToExpression;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.LiteralExpression;
+import org.carbondata.scan.expression.conditional.EqualToExpression;
 
 import junit.framework.TestCase;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputMapperTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputMapperTest.java b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputMapperTest.java
index 5fa8169..a20f957 100644
--- a/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputMapperTest.java
+++ b/hadoop/src/test/java/org/carbondata/hadoop/ft/CarbonInputMapperTest.java
@@ -12,11 +12,11 @@ import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.hadoop.CarbonInputFormat;
 import org.carbondata.hadoop.CarbonProjection;
 import org.carbondata.hadoop.test.util.StoreCreator;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.LiteralExpression;
-import org.carbondata.query.expression.conditional.EqualToExpression;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.LiteralExpression;
+import org.carbondata.scan.expression.conditional.EqualToExpression;
 
 import junit.framework.TestCase;
 import org.apache.hadoop.conf.Configuration;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/hadoop/src/test/java/org/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java b/hadoop/src/test/java/org/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
index 031ea6b..e5e3fb6 100644
--- a/hadoop/src/test/java/org/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
+++ b/hadoop/src/test/java/org/carbondata/hadoop/test/util/ObjectSerializationUtilTest.java
@@ -20,11 +20,11 @@
 package org.carbondata.hadoop.test.util;
 
 import org.carbondata.hadoop.util.ObjectSerializationUtil;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.DataType;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.LiteralExpression;
-import org.carbondata.query.expression.conditional.EqualToExpression;
+import org.carbondata.scan.expression.ColumnExpression;
+import org.carbondata.scan.expression.DataType;
+import org.carbondata.scan.expression.Expression;
+import org.carbondata.scan.expression.LiteralExpression;
+import org.carbondata.scan.expression.conditional.EqualToExpression;
 
 import junit.framework.TestCase;
 import org.junit.Assert;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
index ce0f656..89803f0 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
@@ -37,14 +37,14 @@ import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.iterator.CarbonIterator;
 import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.executor.QueryExecutor;
-import org.carbondata.query.carbon.executor.QueryExecutorFactory;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.model.QueryDimension;
-import org.carbondata.query.carbon.model.QueryMeasure;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchRawResult;
-import org.carbondata.query.carbon.result.iterator.RawResultIterator;
+import org.carbondata.scan.executor.QueryExecutor;
+import org.carbondata.scan.executor.QueryExecutorFactory;
+import org.carbondata.scan.executor.exception.QueryExecutionException;
+import org.carbondata.scan.model.QueryDimension;
+import org.carbondata.scan.model.QueryMeasure;
+import org.carbondata.scan.model.QueryModel;
+import org.carbondata.scan.result.BatchRawResult;
+import org.carbondata.scan.result.iterator.RawResultIterator;
 
 /**
  * Executor class for executing the query on the selected segments to be merged.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
index 66ec1d4..53d32b7 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
@@ -50,8 +50,8 @@ import org.carbondata.processing.store.CarbonFactDataHandlerColumnar;
 import org.carbondata.processing.store.CarbonFactDataHandlerModel;
 import org.carbondata.processing.store.CarbonFactHandler;
 import org.carbondata.processing.store.writer.exception.CarbonDataWriterException;
-import org.carbondata.query.carbon.result.iterator.RawResultIterator;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
+import org.carbondata.scan.result.iterator.RawResultIterator;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
 import org.carbondata.spark.load.CarbonLoadModel;
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/TupleConversionAdapter.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/TupleConversionAdapter.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/TupleConversionAdapter.java
index 593afc6..d5c4d6a 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/TupleConversionAdapter.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/TupleConversionAdapter.java
@@ -24,7 +24,7 @@ import java.util.List;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
 import org.carbondata.processing.util.RemoveDictionaryUtil;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
+import org.carbondata.scan.wrappers.ByteArrayWrapper;
 
 /**
  * This class will be used to convert the Result into the format used in data writer.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/spark/partition/api/DataPartitioner.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/partition/api/DataPartitioner.java b/integration/spark/src/main/java/org/carbondata/spark/partition/api/DataPartitioner.java
index a5d8bf4..1e87ac8 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/partition/api/DataPartitioner.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/partition/api/DataPartitioner.java
@@ -30,7 +30,7 @@ package org.carbondata.spark.partition.api;
 
 import java.util.List;
 
-import org.carbondata.query.carbon.model.CarbonQueryPlan;
+import org.carbondata.scan.model.CarbonQueryPlan;
 
 import org.apache.spark.sql.execution.command.Partitioner;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
index cb59d91..c14515b 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
@@ -40,7 +40,7 @@ import java.util.Properties;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbon.model.CarbonQueryPlan;
+import org.carbondata.scan.model.CarbonQueryPlan;
 import org.carbondata.spark.partition.api.DataPartitioner;
 import org.carbondata.spark.partition.api.Partition;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
index cae2f28..11ad382 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/SampleDataPartitionerImpl.java
@@ -26,7 +26,7 @@ import java.util.List;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbon.model.CarbonQueryPlan;
+import org.carbondata.scan.model.CarbonQueryPlan;
 import org.carbondata.spark.partition.api.DataPartitioner;
 import org.carbondata.spark.partition.api.Partition;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java b/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
index 1e4d877..85a3418 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/readsupport/SparkRowReadSupportImpl.java
@@ -5,7 +5,7 @@ import java.sql.Timestamp;
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
 import org.carbondata.hadoop.readsupport.impl.AbstractDictionaryDecodedReadSupport;
-import org.carbondata.query.carbon.util.DataTypeUtil;
+import org.carbondata.scan.util.DataTypeUtil;
 
 import org.apache.spark.sql.Row;
 import org.apache.spark.sql.catalyst.expressions.GenericRow;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java b/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
index 4b997b5..b9d5fce 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
@@ -34,7 +34,7 @@ import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.carbondata.core.load.LoadMetadataDetails;
 import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.carbon.model.CarbonQueryPlan;
+import org.carbondata.scan.model.CarbonQueryPlan;
 import org.carbondata.spark.partition.api.Partition;
 import org.carbondata.spark.partition.api.impl.DefaultLoadBalancer;
 import org.carbondata.spark.partition.api.impl.PartitionMultiFileImpl;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
index 2dd569b..3b52897 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala
@@ -22,7 +22,7 @@ import org.apache.spark.sql.catalyst.expressions.{Attribute, ExprId, LeafExpress
 import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.types.DataType
 
-import org.carbondata.query.expression.ColumnExpression
+import org.carbondata.scan.expression.ColumnExpression
 
 
 case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nullable: Boolean)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
index da5ecf1..06f05d3 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceHadoopRelation.scala
@@ -39,7 +39,7 @@ import org.apache.spark.util.SerializableConfiguration
 
 import org.carbondata.core.carbon.CarbonTableIdentifier
 import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit, CarbonProjection}
-import org.carbondata.query.expression.logical.AndExpression
+import org.carbondata.scan.expression.logical.AndExpression
 import org.carbondata.spark.{CarbonFilters, CarbonOption}
 import org.carbondata.spark.readsupport.SparkRowReadSupportImpl
 import org.carbondata.spark.util.CarbonScalaUtil.CarbonSparkUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 5359009..c2e1564 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -33,7 +33,7 @@ import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifie
 import org.carbondata.core.carbon.metadata.datatype.DataType
 import org.carbondata.core.carbon.metadata.encoder.Encoding
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
-import org.carbondata.query.carbon.util.DataTypeUtil
+import org.carbondata.scan.util.DataTypeUtil
 
 /**
  * It decodes the data.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index 22fa4fb..6b2bdc2 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -32,7 +32,7 @@ import org.apache.spark.unsafe.types.UTF8String
 
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.util.CarbonProperties
-import org.carbondata.query.carbon.model._
+import org.carbondata.scan.model._
 import org.carbondata.spark.{CarbonFilters, RawValue, RawValueImpl}
 import org.carbondata.spark.rdd.CarbonScanRDD
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala b/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
index 35e1035..e8f6e11 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/SparkUnknownExpression.scala
@@ -24,10 +24,10 @@ import scala.collection.JavaConverters._
 import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression, GenericMutableRow}
 
 import org.carbondata.core.carbon.metadata.encoder.Encoding
-import org.carbondata.query.carbonfilterinterface.{ExpressionType, RowIntf}
-import org.carbondata.query.expression.{ColumnExpression, ExpressionResult, UnknownExpression}
-import org.carbondata.query.expression.conditional.ConditionalExpression
-import org.carbondata.query.expression.exception.FilterUnsupportedException
+import org.carbondata.scan.expression.{ColumnExpression, ExpressionResult, UnknownExpression}
+import org.carbondata.scan.expression.conditional.ConditionalExpression
+import org.carbondata.scan.expression.exception.FilterUnsupportedException
+import org.carbondata.scan.filter.intf.{ExpressionType, RowIntf}
 import org.carbondata.spark.util.CarbonScalaUtil
 
 class SparkUnknownExpression(sparkExp: SparkExpression)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
index cf31a7b..e02bc7f 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/CarbonFilters.scala
@@ -30,9 +30,9 @@ import org.apache.spark.sql.types.StructType
 import org.carbondata.core.carbon.metadata.datatype.DataType
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn
-import org.carbondata.query.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
-import org.carbondata.query.expression.conditional._
-import org.carbondata.query.expression.logical.{AndExpression, OrExpression}
+import org.carbondata.scan.expression.{ColumnExpression => CarbonColumnExpression, Expression => CarbonExpression, LiteralExpression => CarbonLiteralExpression}
+import org.carbondata.scan.expression.conditional._
+import org.carbondata.scan.expression.logical.{AndExpression, OrExpression}
 import org.carbondata.spark.util.CarbonScalaUtil
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
index dd2a10a..e18b64d 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonMergerRDD.scala
@@ -36,7 +36,7 @@ import org.carbondata.core.util.CarbonProperties
 import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
 import org.carbondata.integration.spark.merger.{CarbonCompactionExecutor, CarbonCompactionUtil,
 RowResultMerger}
-import org.carbondata.query.carbon.result.iterator.RawResultIterator
+import org.carbondata.scan.result.iterator.RawResultIterator
 import org.carbondata.spark.MergeResult
 import org.carbondata.spark.load.{CarbonLoaderUtil, CarbonLoadModel}
 import org.carbondata.spark.merger.CarbonDataMergerUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
index 6693108..84a362b 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
@@ -15,7 +15,6 @@
  * limitations under the License.
  */
 
-
 package org.carbondata.spark.rdd
 
 import java.util
@@ -32,11 +31,11 @@ import org.carbondata.common.logging.LogServiceFactory
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo
 import org.carbondata.core.iterator.CarbonIterator
 import org.carbondata.hadoop.{CarbonInputFormat, CarbonInputSplit}
-import org.carbondata.query.carbon.executor.QueryExecutorFactory
-import org.carbondata.query.carbon.model.QueryModel
-import org.carbondata.query.carbon.result.BatchResult
-import org.carbondata.query.carbon.result.iterator.ChunkRowIterator
-import org.carbondata.query.expression.Expression
+import org.carbondata.scan.executor.QueryExecutorFactory
+import org.carbondata.scan.expression.Expression
+import org.carbondata.scan.model.QueryModel
+import org.carbondata.scan.result.BatchResult
+import org.carbondata.scan.result.iterator.ChunkRowIterator
 import org.carbondata.spark.RawValue
 import org.carbondata.spark.load.CarbonLoaderUtil
 import org.carbondata.spark.util.QueryPlanUtil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
index 9bc1a64..593aadb 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/util/CarbonScalaUtil.scala
@@ -28,7 +28,7 @@ import org.carbondata.core.carbon.metadata.datatype.DataType
 import org.carbondata.core.carbon.metadata.encoder.Encoding
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable
 import org.carbondata.core.constants.CarbonCommonConstants
-import org.carbondata.query.expression.{DataType => CarbonDataType}
+import org.carbondata.scan.expression.{DataType => CarbonDataType}
 
 object CarbonScalaUtil {
   def convertSparkToCarbonDataType(


[38/56] [abbrv] incubator-carbondata git commit: Refactor org.carbondata.query package (#692)

Posted by jb...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java b/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
deleted file mode 100644
index 92334db..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/QueryModel.java
+++ /dev/null
@@ -1,516 +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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.Map;
-
-import org.carbondata.core.cache.dictionary.Dictionary;
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
-import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonColumn;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
-import org.carbondata.core.util.CarbonUtil;
-import org.carbondata.query.expression.ColumnExpression;
-import org.carbondata.query.expression.Expression;
-import org.carbondata.query.expression.UnknownExpression;
-import org.carbondata.query.expression.conditional.ConditionalExpression;
-import org.carbondata.query.filter.resolver.FilterResolverIntf;
-
-/**
- * Query model which will have all the detail
- * about the query, This will be sent from driver to executor '
- * This will be refereed to executing the query.
- */
-public class QueryModel implements Serializable {
-
-  /**
-   * serialization version
-   */
-  private static final long serialVersionUID = -4674677234007089052L;
-
-  /**
-   * list of dimension selected for in query
-   */
-  private List<QueryDimension> queryDimension;
-
-  /**
-   * list of dimension in which sorting is applied
-   */
-  private List<QueryDimension> sortDimension;
-
-  /**
-   * list of measure selected in query
-   */
-  private List<QueryMeasure> queryMeasures;
-
-  /**
-   * query id
-   */
-  private String queryId;
-
-  /**
-   * to check if it a aggregate table
-   */
-  private boolean isAggTable;
-
-  /**
-   * filter tree
-   */
-  private FilterResolverIntf filterExpressionResolverTree;
-
-  /**
-   * in case of lime query we need to know how many
-   * records will passed from executor
-   */
-  private int limit;
-
-  /**
-   * to check if it is a count star query , so processing will be different
-   */
-  private boolean isCountStarQuery;
-
-  /**
-   * to check whether aggregation is required during query execution
-   */
-  private boolean detailQuery;
-
-  /**
-   * table block information in which query will be executed
-   */
-  private List<TableBlockInfo> tableBlockInfos;
-
-  /**
-   * sort in which dimension will be get sorted
-   */
-  private byte[] sortOrder;
-
-  /**
-   * absolute table identifier
-   */
-  private AbsoluteTableIdentifier absoluteTableIdentifier;
-  /**
-   * in case of detail query with sort we are spilling to disk
-   * to this location will be used to write the temp file in this location
-   */
-  private String queryTempLocation;
-
-  /**
-   * To handle most of the computation in query engines like spark and hive, carbon should give
-   * raw detailed records to it.
-   */
-  private boolean forcedDetailRawQuery;
-
-  /**
-   * paritition column list
-   */
-  private List<String> paritionColumns;
-
-  /**
-   * this will hold the information about the dictionary dimension
-   * which to
-   */
-  public transient Map<String, Dictionary> columnToDictionaryMapping;
-
-  /**
-   * table on which query will be executed
-   * TODO need to remove this ad pass only the path
-   * and carbon metadata will load the table from metadata file
-   */
-  private CarbonTable table;
-
-  /**
-   * This is used only whne [forcedDetailRawQuery = true]. By default forcedDetailRawQuery returns
-   * dictionary values. But user wants in detail raw bytes the user set this field to true.
-   */
-  private boolean rawBytesDetailQuery;
-
-  public QueryModel() {
-    tableBlockInfos = new ArrayList<TableBlockInfo>();
-    queryDimension = new ArrayList<QueryDimension>();
-    queryMeasures = new ArrayList<QueryMeasure>();
-    sortDimension = new ArrayList<QueryDimension>();
-    sortOrder = new byte[0];
-    paritionColumns = new ArrayList<String>();
-
-  }
-
-  public static QueryModel createModel(AbsoluteTableIdentifier absoluteTableIdentifier,
-      CarbonQueryPlan queryPlan, CarbonTable carbonTable) {
-    QueryModel queryModel = new QueryModel();
-    String factTableName = carbonTable.getFactTableName();
-    queryModel.setAbsoluteTableIdentifier(absoluteTableIdentifier);
-
-    fillQueryModel(queryPlan, carbonTable, queryModel, factTableName);
-
-    queryModel.setLimit(queryPlan.getLimit());
-    queryModel.setDetailQuery(queryPlan.isDetailQuery());
-    queryModel.setForcedDetailRawQuery(queryPlan.isRawDetailQuery());
-    queryModel.setQueryId(queryPlan.getQueryId());
-    queryModel.setQueryTempLocation(queryPlan.getOutLocationPath());
-    return queryModel;
-  }
-
-  private static void fillQueryModel(CarbonQueryPlan queryPlan, CarbonTable carbonTable,
-      QueryModel queryModel, String factTableName) {
-    queryModel.setAbsoluteTableIdentifier(carbonTable.getAbsoluteTableIdentifier());
-    queryModel.setQueryDimension(queryPlan.getDimensions());
-    fillSortInfoInModel(queryModel, queryPlan.getSortedDimemsions());
-    queryModel.setQueryMeasures(
-        queryPlan.getMeasures());
-    if (null != queryPlan.getFilterExpression()) {
-      processFilterExpression(queryPlan.getFilterExpression(),
-          carbonTable.getDimensionByTableName(factTableName),
-          carbonTable.getMeasureByTableName(factTableName));
-    }
-    queryModel.setCountStarQuery(queryPlan.isCountStarQuery());
-    //TODO need to remove this code, and executor will load the table
-    // from file metadata
-    queryModel.setTable(carbonTable);
-  }
-
-  private static void fillSortInfoInModel(QueryModel executorModel,
-      List<QueryDimension> sortedDims) {
-    if (null != sortedDims) {
-      byte[] sortOrderByteArray = new byte[sortedDims.size()];
-      int i = 0;
-      for (QueryColumn mdim : sortedDims) {
-        sortOrderByteArray[i++] = (byte) mdim.getSortOrder().ordinal();
-      }
-      executorModel.setSortOrder(sortOrderByteArray);
-      executorModel.setSortDimension(sortedDims);
-    } else {
-      executorModel.setSortOrder(new byte[0]);
-      executorModel.setSortDimension(new ArrayList<QueryDimension>(0));
-    }
-
-  }
-
-  public static void processFilterExpression(
-      Expression filterExpression, List<CarbonDimension> dimensions, List<CarbonMeasure> measures) {
-    if (null != filterExpression) {
-      if (null != filterExpression.getChildren() && filterExpression.getChildren().size() == 0) {
-        if (filterExpression instanceof ConditionalExpression) {
-          List<ColumnExpression> listOfCol =
-              ((ConditionalExpression) filterExpression).getColumnList();
-          for (ColumnExpression expression : listOfCol) {
-            setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression);
-          }
-
-        }
-      }
-      for (Expression expression : filterExpression.getChildren()) {
-
-        if (expression instanceof ColumnExpression) {
-          setDimAndMsrColumnNode(dimensions, measures, (ColumnExpression) expression);
-        } else if (expression instanceof UnknownExpression) {
-          UnknownExpression exp = ((UnknownExpression) expression);
-          List<ColumnExpression> listOfColExpression = exp.getAllColumnList();
-          for (ColumnExpression col : listOfColExpression) {
-            setDimAndMsrColumnNode(dimensions, measures, col);
-          }
-        } else {
-          processFilterExpression(expression, dimensions, measures);
-        }
-      }
-    }
-
-  }
-
-  private static CarbonMeasure getCarbonMetadataMeasure(String name, List<CarbonMeasure> measures) {
-    for (CarbonMeasure measure : measures) {
-      if (measure.getColName().equalsIgnoreCase(name)) {
-        return measure;
-      }
-    }
-    return null;
-  }
-
-  private static void setDimAndMsrColumnNode(List<CarbonDimension> dimensions,
-      List<CarbonMeasure> measures, ColumnExpression col) {
-    CarbonDimension dim;
-    CarbonMeasure msr;
-    String columnName;
-    columnName = col.getColumnName();
-    dim = CarbonUtil.findDimension(dimensions, columnName);
-    col.setCarbonColumn(dim);
-    col.setDimension(dim);
-    col.setDimension(true);
-    if (null == dim) {
-      msr = getCarbonMetadataMeasure(columnName, measures);
-      col.setCarbonColumn(msr);
-      col.setDimension(false);
-    }
-  }
-
-  /**
-   * It gets the projection columns
-   */
-  public CarbonColumn[] getProjectionColumns() {
-    CarbonColumn[] carbonColumns =
-        new CarbonColumn[getQueryDimension().size() + getQueryMeasures()
-            .size()];
-    for (QueryDimension dimension : getQueryDimension()) {
-      carbonColumns[dimension.getQueryOrder()] = dimension.getDimension();
-    }
-    for (QueryMeasure msr : getQueryMeasures()) {
-      carbonColumns[msr.getQueryOrder()] = msr.getMeasure();
-    }
-    return carbonColumns;
-  }
-
-  /**
-   * @return the queryDimension
-   */
-  public List<QueryDimension> getQueryDimension() {
-    return queryDimension;
-  }
-
-  /**
-   * @param queryDimension the queryDimension to set
-   */
-  public void setQueryDimension(List<QueryDimension> queryDimension) {
-    this.queryDimension = queryDimension;
-  }
-
-  /**
-   * @return the queryMeasures
-   */
-  public List<QueryMeasure> getQueryMeasures() {
-    return queryMeasures;
-  }
-
-  /**
-   * @param queryMeasures the queryMeasures to set
-   */
-  public void setQueryMeasures(List<QueryMeasure> queryMeasures) {
-    this.queryMeasures = queryMeasures;
-  }
-
-  /**
-   * @return the queryId
-   */
-  public String getQueryId() {
-    return queryId;
-  }
-
-  /**
-   * @param queryId the queryId to set
-   */
-  public void setQueryId(String queryId) {
-    this.queryId = queryId;
-  }
-
-  /**
-   * @return the isAggTable
-   */
-  public boolean isAggTable() {
-    return isAggTable;
-  }
-
-  /**
-   * @param isAggTable the isAggTable to set
-   */
-  public void setAggTable(boolean isAggTable) {
-    this.isAggTable = isAggTable;
-  }
-
-  /**
-   * @return the limit
-   */
-  public int getLimit() {
-    return limit;
-  }
-
-  /**
-   * @param limit the limit to set
-   */
-  public void setLimit(int limit) {
-    this.limit = limit;
-  }
-
-  /**
-   * @return the isCountStarQuery
-   */
-  public boolean isCountStarQuery() {
-    return isCountStarQuery;
-  }
-
-  /**
-   * @param isCountStarQuery the isCountStarQuery to set
-   */
-  public void setCountStarQuery(boolean isCountStarQuery) {
-    this.isCountStarQuery = isCountStarQuery;
-  }
-
-  /**
-   * @return the isdetailQuery
-   */
-  public boolean isDetailQuery() {
-    return detailQuery;
-  }
-
-  public void setDetailQuery(boolean detailQuery) {
-    this.detailQuery = detailQuery;
-  }
-
-  /**
-   * @return the tableBlockInfos
-   */
-  public List<TableBlockInfo> getTableBlockInfos() {
-    return tableBlockInfos;
-  }
-
-  /**
-   * @param tableBlockInfos the tableBlockInfos to set
-   */
-  public void setTableBlockInfos(List<TableBlockInfo> tableBlockInfos) {
-    this.tableBlockInfos = tableBlockInfos;
-  }
-
-  /**
-   * @return the queryTempLocation
-   */
-  public String getQueryTempLocation() {
-    return queryTempLocation;
-  }
-
-  /**
-   * @param queryTempLocation the queryTempLocation to set
-   */
-  public void setQueryTempLocation(String queryTempLocation) {
-    this.queryTempLocation = queryTempLocation;
-  }
-
-  /**
-   * @return the sortOrder
-   */
-  public byte[] getSortOrder() {
-    return sortOrder;
-  }
-
-  /**
-   * @param sortOrder the sortOrder to set
-   */
-  public void setSortOrder(byte[] sortOrder) {
-    this.sortOrder = sortOrder;
-  }
-
-  /**
-   * @return the sortDimension
-   */
-  public List<QueryDimension> getSortDimension() {
-    return sortDimension;
-  }
-
-  /**
-   * @param sortDimension the sortDimension to set
-   */
-  public void setSortDimension(List<QueryDimension> sortDimension) {
-    this.sortDimension = sortDimension;
-  }
-
-  /**
-   * @return the filterEvaluatorTree
-   */
-  public FilterResolverIntf getFilterExpressionResolverTree() {
-    return filterExpressionResolverTree;
-  }
-
-  public void setFilterExpressionResolverTree(FilterResolverIntf filterExpressionResolverTree) {
-    this.filterExpressionResolverTree = filterExpressionResolverTree;
-  }
-
-  /**
-   * @return the absoluteTableIdentifier
-   */
-  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
-    return absoluteTableIdentifier;
-  }
-
-  /**
-   * @param absoluteTableIdentifier the absoluteTableIdentifier to set
-   */
-  public void setAbsoluteTableIdentifier(AbsoluteTableIdentifier absoluteTableIdentifier) {
-    this.absoluteTableIdentifier = absoluteTableIdentifier;
-  }
-
-  /**
-   * @return the paritionColumns
-   */
-  public List<String> getParitionColumns() {
-    return paritionColumns;
-  }
-
-  /**
-   * @param paritionColumns the paritionColumns to set
-   */
-  public void setParitionColumns(List<String> paritionColumns) {
-    this.paritionColumns = paritionColumns;
-  }
-
-  /**
-   * @return the table
-   */
-  public CarbonTable getTable() {
-    return table;
-  }
-
-  /**
-   * @param table the table to set
-   */
-  public void setTable(CarbonTable table) {
-    this.table = table;
-  }
-
-  public boolean isForcedDetailRawQuery() {
-    return forcedDetailRawQuery;
-  }
-
-  public void setForcedDetailRawQuery(boolean forcedDetailRawQuery) {
-    this.forcedDetailRawQuery = forcedDetailRawQuery;
-  }
-
-  /**
-   * @return
-   */
-  public Map<String, Dictionary> getColumnToDictionaryMapping() {
-    return columnToDictionaryMapping;
-  }
-
-  /**
-   * @param columnToDictionaryMapping
-   */
-  public void setColumnToDictionaryMapping(Map<String, Dictionary> columnToDictionaryMapping) {
-    this.columnToDictionaryMapping = columnToDictionaryMapping;
-  }
-
-  public boolean isRawBytesDetailQuery() {
-    return rawBytesDetailQuery;
-  }
-
-  public void setRawBytesDetailQuery(boolean rawBytesDetailQuery) {
-    this.rawBytesDetailQuery = rawBytesDetailQuery;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/QuerySchemaInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/QuerySchemaInfo.java b/core/src/main/java/org/carbondata/query/carbon/model/QuerySchemaInfo.java
deleted file mode 100644
index 2d1971d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/QuerySchemaInfo.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.carbondata.query.carbon.model;
-
-import java.io.Serializable;
-
-import org.carbondata.core.keygenerator.KeyGenerator;
-
-public class QuerySchemaInfo implements Serializable {
-
-  private int[] maskedByteIndexes;
-
-  private KeyGenerator keyGenerator;
-
-  private QueryDimension[] queryDimensions;
-
-  private QueryMeasure[] queryMeasures;
-
-  private int[] queryOrder;
-
-  private int[] queryReverseOrder;
-
-  public int[] getMaskedByteIndexes() {
-    return maskedByteIndexes;
-  }
-
-  public void setMaskedByteIndexes(int[] maskedByteIndexes) {
-    this.maskedByteIndexes = maskedByteIndexes;
-  }
-
-  public KeyGenerator getKeyGenerator() {
-    return keyGenerator;
-  }
-
-  public void setKeyGenerator(KeyGenerator keyGenerator) {
-    this.keyGenerator = keyGenerator;
-  }
-
-  public QueryDimension[] getQueryDimensions() {
-    return queryDimensions;
-  }
-
-  public void setQueryDimensions(QueryDimension[] queryDimensions) {
-    this.queryDimensions = queryDimensions;
-  }
-
-  public QueryMeasure[] getQueryMeasures() {
-    return queryMeasures;
-  }
-
-  public void setQueryMeasures(QueryMeasure[] queryMeasures) {
-    this.queryMeasures = queryMeasures;
-  }
-
-  public int[] getQueryOrder() {
-    return queryOrder;
-  }
-
-  public void setQueryOrder(int[] queryOrder) {
-    this.queryOrder = queryOrder;
-  }
-
-  public int[] getQueryReverseOrder() {
-    return queryReverseOrder;
-  }
-
-  public void setQueryReverseOrder(int[] queryReverseOrder) {
-    this.queryReverseOrder = queryReverseOrder;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/model/SortOrderType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/model/SortOrderType.java b/core/src/main/java/org/carbondata/query/carbon/model/SortOrderType.java
deleted file mode 100644
index a734a46..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/model/SortOrderType.java
+++ /dev/null
@@ -1,57 +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.carbondata.query.carbon.model;
-
-/**
- * enum for sorting the columns
- */
-public enum SortOrderType {
-
-    /**
-     * Ascending order
-     */
-    ASC(0),
-
-    /**
-     * Descending order.
-     */
-    DSC(1),
-
-    /**
-     * No order mentioned
-     */
-    NONE(-1);
-  /**
-   * Order type in numeric
-   */
-  private int orderType;
-
-  SortOrderType(int orderType) {
-    this.orderType = orderType;
-  }
-
-  /**
-   * Order type in number
-   *
-   * @return orderType int
-   */
-  public int getOrderType() {
-    return orderType;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java b/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java
deleted file mode 100644
index 52b1bdf..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/AbstractDataBlockIterator.java
+++ /dev/null
@@ -1,126 +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.carbondata.query.carbon.processor;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.collector.ScannedResultCollector;
-import org.carbondata.query.carbon.collector.impl.ListBasedResultCollector;
-import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.scanner.BlockletScanner;
-import org.carbondata.query.carbon.scanner.impl.FilterScanner;
-import org.carbondata.query.carbon.scanner.impl.NonFilterScanner;
-
-/**
- * This abstract class provides a skeletal implementation of the
- * Block iterator.
- */
-public abstract class AbstractDataBlockIterator extends CarbonIterator<Result> {
-
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractDataBlockIterator.class.getName());
-  /**
-   * iterator which will be used to iterate over data blocks
-   */
-  protected CarbonIterator<DataRefNode> dataBlockIterator;
-
-  /**
-   * execution details
-   */
-  protected BlockExecutionInfo blockExecutionInfo;
-
-  /**
-   * result collector which will be used to aggregate the scanned result
-   */
-  protected ScannedResultCollector scannerResultAggregator;
-
-  /**
-   * processor which will be used to process the block processing can be
-   * filter processing or non filter processing
-   */
-  protected BlockletScanner blockletScanner;
-
-  /**
-   * to hold the data block
-   */
-  protected BlocksChunkHolder blocksChunkHolder;
-
-  /**
-   * batch size of result
-   */
-  protected int batchSize;
-
-  protected AbstractScannedResult scannedResult;
-
-  public AbstractDataBlockIterator(BlockExecutionInfo blockExecutionInfo,
-      FileHolder fileReader, int batchSize) {
-    this.blockExecutionInfo = blockExecutionInfo;
-    dataBlockIterator = new BlockletIterator(blockExecutionInfo.getFirstDataBlock(),
-        blockExecutionInfo.getNumberOfBlockToScan());
-    blocksChunkHolder = new BlocksChunkHolder(blockExecutionInfo.getTotalNumberDimensionBlock(),
-        blockExecutionInfo.getTotalNumberOfMeasureBlock());
-    blocksChunkHolder.setFileReader(fileReader);
-
-    if (blockExecutionInfo.getFilterExecuterTree() != null) {
-      blockletScanner = new FilterScanner(blockExecutionInfo);
-    } else {
-      blockletScanner = new NonFilterScanner(blockExecutionInfo);
-    }
-
-    this.scannerResultAggregator =
-        new ListBasedResultCollector(blockExecutionInfo);
-    this.batchSize = batchSize;
-  }
-
-  public boolean hasNext() {
-    try {
-      if (scannedResult != null && scannedResult.hasNext()) {
-        return true;
-      } else {
-        scannedResult = getNextScannedResult();
-        while (scannedResult != null) {
-          if (scannedResult.hasNext()) {
-            return true;
-          }
-          scannedResult = getNextScannedResult();
-        }
-        return false;
-      }
-    } catch (QueryExecutionException ex) {
-      throw new RuntimeException(ex);
-    }
-  }
-
-  private AbstractScannedResult getNextScannedResult() throws QueryExecutionException {
-    if (dataBlockIterator.hasNext()) {
-      blocksChunkHolder.setDataBlock(dataBlockIterator.next());
-      blocksChunkHolder.reset();
-      return blockletScanner.scanBlocklet(blocksChunkHolder);
-    }
-    return null;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/processor/BlockletIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/BlockletIterator.java b/core/src/main/java/org/carbondata/query/carbon/processor/BlockletIterator.java
deleted file mode 100644
index 307ff64..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/BlockletIterator.java
+++ /dev/null
@@ -1,88 +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.carbondata.query.carbon.processor;
-
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.iterator.CarbonIterator;
-
-/**
- * Below class will be used to iterate over data block
- */
-public class BlockletIterator extends CarbonIterator<DataRefNode> {
-  /**
-   * data store block
-   */
-  protected DataRefNode datablock;
-  /**
-   * block counter to keep a track how many block has been processed
-   */
-  private int blockCounter;
-
-  /**
-   * flag to be used to check any more data block is present or not
-   */
-  private boolean hasNext = true;
-
-  /**
-   * total number blocks assgned to this iterator
-   */
-  private long totalNumberOfBlocksToScan;
-
-  /**
-   * Constructor
-   *
-   * @param datablock                 first data block
-   * @param totalNumberOfBlocksToScan total number of blocks to be scanned
-   */
-  public BlockletIterator(DataRefNode datablock, long totalNumberOfBlocksToScan) {
-    this.datablock = datablock;
-    this.totalNumberOfBlocksToScan = totalNumberOfBlocksToScan;
-  }
-
-  /**
-   * is all the blocks assigned to this iterator has been processed
-   */
-  @Override public boolean hasNext() {
-    return hasNext;
-  }
-
-  @Override
-  /**
-   * To get the next block
-   * @return next data block
-   *
-   */
-  public DataRefNode next() {
-    // get the current blocks
-    DataRefNode datablockTemp = datablock;
-    // store the next data block
-    datablock = datablock.getNextDataRefNode();
-    // increment the counter
-    blockCounter++;
-    // if all the data block is processed then
-    // set the has next flag to false
-    // or if number of blocks assigned to this iterator is processed
-    // then also set the hasnext flag to false
-    if (null == datablock || blockCounter >= this.totalNumberOfBlocksToScan) {
-      hasNext = false;
-    }
-    return datablockTemp;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/processor/BlocksChunkHolder.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/BlocksChunkHolder.java b/core/src/main/java/org/carbondata/query/carbon/processor/BlocksChunkHolder.java
deleted file mode 100644
index 35ab66d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/BlocksChunkHolder.java
+++ /dev/null
@@ -1,125 +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.carbondata.query.carbon.processor;
-
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.core.datastorage.store.FileHolder;
-
-/**
- * Block chunk holder which will hold the dimension and
- * measure chunk
- */
-public class BlocksChunkHolder {
-
-  /**
-   * dimension column data chunk
-   */
-  private DimensionColumnDataChunk[] dimensionDataChunk;
-
-  /**
-   * measure column data chunk
-   */
-  private MeasureColumnDataChunk[] measureDataChunk;
-
-  /**
-   * file reader which will use to read the block from file
-   */
-  private FileHolder fileReader;
-
-  /**
-   * data block
-   */
-  private DataRefNode dataBlock;
-
-  public BlocksChunkHolder(int numberOfDimensionBlock, int numberOfMeasureBlock) {
-    dimensionDataChunk = new DimensionColumnDataChunk[numberOfDimensionBlock];
-    measureDataChunk = new MeasureColumnDataChunk[numberOfMeasureBlock];
-  }
-
-  /**
-   * @return the dimensionDataChunk
-   */
-  public DimensionColumnDataChunk[] getDimensionDataChunk() {
-    return dimensionDataChunk;
-  }
-
-  /**
-   * @param dimensionDataChunk the dimensionDataChunk to set
-   */
-  public void setDimensionDataChunk(DimensionColumnDataChunk[] dimensionDataChunk) {
-    this.dimensionDataChunk = dimensionDataChunk;
-  }
-
-  /**
-   * @return the measureDataChunk
-   */
-  public MeasureColumnDataChunk[] getMeasureDataChunk() {
-    return measureDataChunk;
-  }
-
-  /**
-   * @param measureDataChunk the measureDataChunk to set
-   */
-  public void setMeasureDataChunk(MeasureColumnDataChunk[] measureDataChunk) {
-    this.measureDataChunk = measureDataChunk;
-  }
-
-  /**
-   * @return the fileReader
-   */
-  public FileHolder getFileReader() {
-    return fileReader;
-  }
-
-  /**
-   * @param fileReader the fileReader to set
-   */
-  public void setFileReader(FileHolder fileReader) {
-    this.fileReader = fileReader;
-  }
-
-  /**
-   * @return the dataBlock
-   */
-  public DataRefNode getDataBlock() {
-    return dataBlock;
-  }
-
-  /**
-   * @param dataBlock the dataBlock to set
-   */
-  public void setDataBlock(DataRefNode dataBlock) {
-    this.dataBlock = dataBlock;
-  }
-
-  /***
-   * To reset the measure chunk and dimension chunk
-   * array
-   */
-  public void reset() {
-    for (int i = 0; i < measureDataChunk.length; i++) {
-      this.measureDataChunk[i] = null;
-    }
-    for (int i = 0; i < dimensionDataChunk.length; i++) {
-      this.dimensionDataChunk[i] = null;
-    }
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.java b/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.java
deleted file mode 100644
index ade1965..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/processor/impl/DataBlockIteratorImpl.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.carbondata.query.carbon.processor.impl;
-
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.processor.AbstractDataBlockIterator;
-import org.carbondata.query.carbon.result.Result;
-
-/**
- * Below class will be used to process the block for detail query
- */
-public class DataBlockIteratorImpl extends AbstractDataBlockIterator {
-
-  /**
-   * DataBlockIteratorImpl Constructor
-   *
-   * @param blockExecutionInfo execution information
-   */
-  public DataBlockIteratorImpl(BlockExecutionInfo blockExecutionInfo,
-      FileHolder fileReader, int batchSize) {
-    super(blockExecutionInfo, fileReader, batchSize);
-  }
-
-  /**
-   * It scans the block and returns the result with @batchSize
-   *
-   * @return Result of @batchSize
-   */
-  public Result next() {
-    this.scannerResultAggregator.collectData(scannedResult, batchSize);
-    Result result = this.scannerResultAggregator.getCollectedResult();
-    while (result.size() < batchSize && hasNext()) {
-      this.scannerResultAggregator.collectData(scannedResult, batchSize-result.size());
-      result.merge(this.scannerResultAggregator.getCollectedResult());
-    }
-    return result;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
deleted file mode 100644
index 444b2bd..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/AbstractScannedResult.java
+++ /dev/null
@@ -1,347 +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.carbondata.query.carbon.result;
-
-import java.math.BigDecimal;
-import java.util.Map;
-
-import org.carbondata.core.carbon.datastore.chunk.DimensionColumnDataChunk;
-import org.carbondata.core.carbon.datastore.chunk.MeasureColumnDataChunk;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
-
-/**
- * Scanned result class which will store and provide the result on request
- */
-public abstract class AbstractScannedResult {
-
-  /**
-   * current row number
-   */
-  protected int currentRow = -1;
-  /**
-   * row mapping indexes
-   */
-  protected int[] rowMapping;
-  /**
-   * key size of the fixed length column
-   */
-  private int fixedLengthKeySize;
-  /**
-   * total number of rows
-   */
-  private int totalNumberOfRows;
-  /**
-   * to keep track of number of rows process
-   */
-  private int rowCounter;
-  /**
-   * dimension column data chunk
-   */
-  private DimensionColumnDataChunk[] dataChunks;
-  /**
-   * measure column data chunk
-   */
-  private MeasureColumnDataChunk[] measureDataChunks;
-  /**
-   * dictionary column block index in file
-   */
-  private int[] dictionaryColumnBlockIndexes;
-
-  /**
-   * no dictionary column block index in file
-   */
-  private int[] noDictionaryColumnBlockIndexes;
-
-  /**
-   * column group to is key structure info
-   * which will be used to get the key from the complete
-   * column group key
-   * For example if only one dimension of the column group is selected
-   * then from complete column group key it will be used to mask the key and
-   * get the particular column key
-   */
-  private Map<Integer, KeyStructureInfo> columnGroupKeyStructureInfo;
-
-  public AbstractScannedResult(BlockExecutionInfo blockExecutionInfo) {
-    this.fixedLengthKeySize = blockExecutionInfo.getFixedLengthKeySize();
-    this.noDictionaryColumnBlockIndexes = blockExecutionInfo.getNoDictionaryBlockIndexes();
-    this.dictionaryColumnBlockIndexes = blockExecutionInfo.getDictionaryColumnBlockIndex();
-    this.columnGroupKeyStructureInfo = blockExecutionInfo.getColumnGroupToKeyStructureInfo();
-  }
-
-  /**
-   * Below method will be used to set the dimension chunks
-   * which will be used to create a row
-   *
-   * @param dataChunks dimension chunks used in query
-   */
-  public void setDimensionChunks(DimensionColumnDataChunk[] dataChunks) {
-    this.dataChunks = dataChunks;
-  }
-
-  /**
-   * Below method will be used to set the measure column chunks
-   *
-   * @param measureDataChunks measure data chunks
-   */
-  public void setMeasureChunks(MeasureColumnDataChunk[] measureDataChunks) {
-    this.measureDataChunks = measureDataChunks;
-  }
-
-  /**
-   * Below method will be used to get the chunk based in measure ordinal
-   *
-   * @param ordinal measure ordinal
-   * @return measure column chunk
-   */
-  public MeasureColumnDataChunk getMeasureChunk(int ordinal) {
-    return measureDataChunks[ordinal];
-  }
-
-  /**
-   * Below method will be used to get the key for all the dictionary dimensions
-   * which is present in the query
-   *
-   * @param rowId row id selected after scanning
-   * @return return the dictionary key
-   */
-  protected byte[] getDictionaryKeyArray(int rowId) {
-    byte[] completeKey = new byte[fixedLengthKeySize];
-    int offset = 0;
-    for (int i = 0; i < this.dictionaryColumnBlockIndexes.length; i++) {
-      offset += dataChunks[dictionaryColumnBlockIndexes[i]]
-          .fillChunkData(completeKey, offset, rowId,
-              columnGroupKeyStructureInfo.get(dictionaryColumnBlockIndexes[i]));
-    }
-    rowCounter++;
-    return completeKey;
-  }
-
-  /**
-   * Just increment the counter incase of query only on measures.
-   */
-  public void incrementCounter() {
-    rowCounter ++;
-    currentRow ++;
-  }
-
-  /**
-   * Below method will be used to get the dimension data based on dimension
-   * ordinal and index
-   *
-   * @param dimOrdinal dimension ordinal present in the query
-   * @param rowId      row index
-   * @return dimension data based on row id
-   */
-  protected byte[] getDimensionData(int dimOrdinal, int rowId) {
-    return dataChunks[dimOrdinal].getChunkData(rowId);
-  }
-
-  /**
-   * Below method will be used to get the dimension key array
-   * for all the no dictionary dimension present in the query
-   *
-   * @param rowId row number
-   * @return no dictionary keys for all no dictionary dimension
-   */
-  protected byte[][] getNoDictionaryKeyArray(int rowId) {
-    byte[][] noDictionaryColumnsKeys = new byte[noDictionaryColumnBlockIndexes.length][];
-    int position = 0;
-    for (int i = 0; i < this.noDictionaryColumnBlockIndexes.length; i++) {
-      noDictionaryColumnsKeys[position++] =
-          dataChunks[noDictionaryColumnBlockIndexes[i]].getChunkData(rowId);
-    }
-    return noDictionaryColumnsKeys;
-  }
-
-  /**
-   * Below method will be used to get the complex type keys array based
-   * on row id for all the complex type dimension selected in query
-   *
-   * @param rowId row number
-   * @return complex type key array for all the complex dimension selected in query
-   */
-  protected byte[][] getComplexTypeKeyArray(int rowId) {
-    return new byte[0][];
-  }
-
-  /**
-   * @return return the total number of row after scanning
-   */
-  public int numberOfOutputRows() {
-    return this.totalNumberOfRows;
-  }
-
-  /**
-   * to check whether any more row is present in the result
-   *
-   * @return
-   */
-  public boolean hasNext() {
-    return rowCounter < this.totalNumberOfRows;
-  }
-
-  /**
-   * As this class will be a flyweight object so
-   * for one block all the blocklet scanning will use same result object
-   * in that case we need to reset the counter to zero so
-   * for new result it will give the result from zero
-   */
-  public void reset() {
-    rowCounter = 0;
-    currentRow = -1;
-  }
-
-  /**
-   * @param totalNumberOfRows set total of number rows valid after scanning
-   */
-  public void setNumberOfRows(int totalNumberOfRows) {
-    this.totalNumberOfRows = totalNumberOfRows;
-  }
-
-  /**
-   * After applying filter it will return the  bit set with the valid row indexes
-   * so below method will be used to set the row indexes
-   *
-   * @param indexes
-   */
-  public void setIndexes(int[] indexes) {
-    this.rowMapping = indexes;
-  }
-
-  /**
-   * Below method will be used to check whether measure value is null or not
-   *
-   * @param ordinal  measure ordinal
-   * @param rowIndex row number to be checked
-   * @return whether it is null or not
-   */
-  protected boolean isNullMeasureValue(int ordinal, int rowIndex) {
-    return measureDataChunks[ordinal].getNullValueIndexHolder().getBitSet().get(rowIndex);
-  }
-
-  /**
-   * Below method will be used to get the measure value of
-   * long type
-   *
-   * @param ordinal  measure ordinal
-   * @param rowIndex row number of the measure value
-   * @return measure value of long type
-   */
-  protected long getLongMeasureValue(int ordinal, int rowIndex) {
-    return measureDataChunks[ordinal].getMeasureDataHolder().getReadableLongValueByIndex(rowIndex);
-  }
-
-  /**
-   * Below method will be used to get the measure value of double type
-   *
-   * @param ordinal  measure ordinal
-   * @param rowIndex row number
-   * @return measure value of double type
-   */
-  protected double getDoubleMeasureValue(int ordinal, int rowIndex) {
-    return measureDataChunks[ordinal].getMeasureDataHolder()
-        .getReadableDoubleValueByIndex(rowIndex);
-  }
-
-  /**
-   * Below method will be used to get the measure type of big decimal data type
-   *
-   * @param ordinal  ordinal of the of the measure
-   * @param rowIndex row number
-   * @return measure of big decimal type
-   */
-  protected BigDecimal getBigDecimalMeasureValue(int ordinal, int rowIndex) {
-    return measureDataChunks[ordinal].getMeasureDataHolder()
-        .getReadableBigDecimalValueByIndex(rowIndex);
-  }
-
-  /**
-   * will return the current valid row id
-   *
-   * @return valid row id
-   */
-  public abstract int getCurrenrRowId();
-
-  /**
-   * @return dictionary key array for all the dictionary dimension
-   * selected in query
-   */
-  public abstract byte[] getDictionaryKeyArray();
-
-  /**
-   * Return the dimension data based on dimension ordinal
-   *
-   * @param dimensionOrdinal dimension ordinal
-   * @return dimension data
-   */
-  public abstract byte[] getDimensionKey(int dimensionOrdinal);
-
-  /**
-   * Below method will be used to get the complex type key array
-   *
-   * @return complex type key array
-   */
-  public abstract byte[][] getComplexTypeKeyArray();
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  public abstract byte[][] getNoDictionaryKeyArray();
-
-  /**
-   * Below method will be used to to check whether measure value
-   * is null or for a measure
-   *
-   * @param ordinal measure ordinal
-   * @return is null or not
-   */
-  public abstract boolean isNullMeasureValue(int ordinal);
-
-  /**
-   * Below method will be used to get the measure value for measure
-   * of long data type
-   *
-   * @param ordinal measure ordinal
-   * @return long value of measure
-   */
-  public abstract long getLongMeasureValue(int ordinal);
-
-  /**
-   * Below method will be used to get the value of measure of double
-   * type
-   *
-   * @param ordinal measure ordinal
-   * @return measure value
-   */
-  public abstract double getDoubleMeasureValue(int ordinal);
-
-  /**
-   * Below method will be used to get the data of big decimal type
-   * of a measure
-   *
-   * @param ordinal measure ordinal
-   * @return measure value
-   */
-  public abstract BigDecimal getBigDecimalMeasureValue(int ordinal);
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java b/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
deleted file mode 100644
index b00c021..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/BatchRawResult.java
+++ /dev/null
@@ -1,43 +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.carbondata.query.carbon.result;
-
-/**
- * Below class holds the query result of batches.
- */
-public class BatchRawResult extends BatchResult {
-
-  /**
-   * This method will return one row at a time based on the counter given.
-   * @param counter
-   * @return
-   */
-  public Object[] getRawRow(int counter) {
-    return rows[counter];
-  }
-
-  /**
-   * For getting the total size.
-   * @return
-   */
-  public int getSize() {
-    return rows.length;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java b/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.java
deleted file mode 100644
index 220336b..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/BatchResult.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.carbondata.query.carbon.result;
-
-import java.util.NoSuchElementException;
-
-import org.carbondata.core.iterator.CarbonIterator;
-
-/**
- * Below class holds the query result
- */
-public class BatchResult extends CarbonIterator<Object[]> {
-
-  /**
-   * list of keys
-   */
-  protected Object[][] rows;
-
-  /**
-   * counter to check whether all the records are processed or not
-   */
-  protected int counter;
-
-  public BatchResult() {
-    this.rows = new Object[0][];
-  }
-
-  /**
-   * Below method will be used to get the rows
-   *
-   * @return
-   */
-  public Object[][] getRows() {
-    return rows;
-  }
-
-  /**
-   * Below method will be used to get the set the values
-   *
-   * @param rows
-   */
-  public void setRows(Object[][] rows) {
-    this.rows = rows;
-  }
-
-
-  /**
-   * Returns {@code true} if the iteration has more elements.
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return counter < rows.length;
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Object[] next() {
-    if (!hasNext()) {
-      throw new NoSuchElementException();
-    }
-    Object[] row = rows[counter];
-    counter++;
-    return row;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java b/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
deleted file mode 100644
index 1a40fc3..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/ListBasedResultWrapper.java
+++ /dev/null
@@ -1,38 +0,0 @@
-package org.carbondata.query.carbon.result;
-
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-public class ListBasedResultWrapper {
-
-  private ByteArrayWrapper key;
-
-  private Object[] value;
-
-  /**
-   * @return the key
-   */
-  public ByteArrayWrapper getKey() {
-    return key;
-  }
-
-  /**
-   * @param key the key to set
-   */
-  public void setKey(ByteArrayWrapper key) {
-    this.key = key;
-  }
-
-  /**
-   * @return the value
-   */
-  public Object[] getValue() {
-    return value;
-  }
-
-  /**
-   * @param value the value to set
-   */
-  public void setValue(Object[] value) {
-    this.value = value;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/Result.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/Result.java b/core/src/main/java/org/carbondata/query/carbon/result/Result.java
deleted file mode 100644
index 8526b07..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/Result.java
+++ /dev/null
@@ -1,70 +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.carbondata.query.carbon.result;
-
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * Result interface for storing the result
- */
-public interface Result<K, V> {
-  /**
-   * Below method will be used to
-   * add the sccaed result
-   *
-   * @param result
-   */
-  void addScannedResult(K result);
-
-  /**
-   * Returns {@code true} if the iteration has more elements.
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  boolean hasNext();
-
-  /**
-   * Below method will return the result key
-   *
-   * @return key
-   */
-  ByteArrayWrapper getKey();
-
-  /**
-   * Below code will return the result value
-   *
-   * @return value
-   */
-  V[] getValue();
-
-  void merge(Result<K, V> otherResult);
-
-  /**
-   * Below method will be used to get the result
-   *
-   * @return
-   */
-  K getResult();
-
-  /**
-   * @return size of the result
-   */
-  int size();
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/comparator/FixedLengthKeyResultComparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/comparator/FixedLengthKeyResultComparator.java b/core/src/main/java/org/carbondata/query/carbon/result/comparator/FixedLengthKeyResultComparator.java
deleted file mode 100644
index cfdcc27..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/comparator/FixedLengthKeyResultComparator.java
+++ /dev/null
@@ -1,71 +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.carbondata.query.carbon.result.comparator;
-
-import java.util.Comparator;
-
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-
-/**
- * Fixed length key comparator
- */
-public class FixedLengthKeyResultComparator implements Comparator<ListBasedResultWrapper> {
-
-  /**
-   * compareRange
-   */
-  private int[] compareRange;
-
-  /**
-   * sortOrder
-   */
-  private byte sortOrder;
-
-  /**
-   * maskedKey
-   */
-  private byte[] maskedKey;
-
-  public FixedLengthKeyResultComparator(int[] compareRange, byte sortOrder, byte[] maskedKey) {
-    this.compareRange = compareRange;
-    this.sortOrder = sortOrder;
-    this.maskedKey = maskedKey;
-  }
-
-  @Override public int compare(ListBasedResultWrapper listBasedResultWrapper1,
-      ListBasedResultWrapper listBasedResultWrapper2) {
-    int cmp = 0;
-    byte[] o1 = listBasedResultWrapper1.getKey().getDictionaryKey();
-    byte[] o2 = listBasedResultWrapper2.getKey().getDictionaryKey();
-    for (int i = 0; i < compareRange.length; i++) {
-      int a = (o1[compareRange[i]] & this.maskedKey[i]) & 0xff;
-      int b = (o2[compareRange[i]] & this.maskedKey[i]) & 0xff;
-      cmp = a - b;
-      if (cmp != 0) {
-
-        if (sortOrder == 1) {
-          return cmp * -1;
-        }
-        return cmp;
-      }
-    }
-    return 0;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/comparator/VariableLengthKeyResultComparator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/comparator/VariableLengthKeyResultComparator.java b/core/src/main/java/org/carbondata/query/carbon/result/comparator/VariableLengthKeyResultComparator.java
deleted file mode 100644
index 262258d..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/comparator/VariableLengthKeyResultComparator.java
+++ /dev/null
@@ -1,80 +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.carbondata.query.carbon.result.comparator;
-
-import java.nio.charset.Charset;
-import java.util.Comparator;
-
-import org.carbondata.core.carbon.metadata.datatype.DataType;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.util.DataTypeUtil;
-
-/**
- * Variable length key comparator
- */
-public class VariableLengthKeyResultComparator implements Comparator<ListBasedResultWrapper> {
-
-  /**
-   * sort order
-   */
-  private byte sortOrder;
-
-  /**
-   * no dictionary column index
-   */
-  private int noDictionaryColumnIndex;
-
-  /**
-   * data type of the column
-   */
-  private DataType dataType;
-
-  public VariableLengthKeyResultComparator(byte sortOrder, int noDictionaryColumnIndex,
-      DataType dataType) {
-    this.sortOrder = sortOrder;
-    this.noDictionaryColumnIndex = noDictionaryColumnIndex;
-    this.dataType = dataType;
-  }
-
-  @Override public int compare(ListBasedResultWrapper listBasedResultWrapperFirst,
-      ListBasedResultWrapper listBasedResultWrapperSecond) {
-    // get the result
-    byte[] noDictionaryKeysFirst =
-        listBasedResultWrapperFirst.getKey().getNoDictionaryKeyByIndex(noDictionaryColumnIndex);
-    // convert the result based on actual data type
-    Object dataBasedOnDataTypeFirst =
-        DataTypeUtil.getDataBasedOnDataType(new String(noDictionaryKeysFirst, Charset
-            .forName(CarbonCommonConstants.DEFAULT_CHARSET)), dataType);
-    byte[] noDictionaryKeysSecond =
-        listBasedResultWrapperSecond.getKey().getNoDictionaryKeyByIndex(noDictionaryColumnIndex);
-    Object dataBasedOnDataTypeSecond = DataTypeUtil.getDataBasedOnDataType(
-        new String(noDictionaryKeysSecond, Charset.forName(CarbonCommonConstants.DEFAULT_CHARSET)),
-        dataType);
-    int cmp = 0;
-    // compare the result
-    cmp = DataTypeUtil
-        .compareBasedOnDatatYpe(dataBasedOnDataTypeFirst, dataBasedOnDataTypeSecond, dataType);
-    if (sortOrder == 1) {
-      cmp = cmp * -1;
-    }
-    return cmp;
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/impl/FilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/impl/FilterQueryScannedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/impl/FilterQueryScannedResult.java
deleted file mode 100644
index 51083a9..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/impl/FilterQueryScannedResult.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.carbondata.query.carbon.result.impl;
-
-import java.math.BigDecimal;
-
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Result provider class in case of filter query
- * In case of filter query data will be send
- * based on filtered row index
- */
-public class FilterQueryScannedResult extends AbstractScannedResult {
-
-  public FilterQueryScannedResult(BlockExecutionInfo tableBlockExecutionInfos) {
-    super(tableBlockExecutionInfos);
-  }
-
-  /**
-   * @return dictionary key array for all the dictionary dimension
-   * selected in query
-   */
-  @Override public byte[] getDictionaryKeyArray() {
-    ++currentRow;
-    return getDictionaryKeyArray(rowMapping[currentRow]);
-  }
-
-  /**
-   * Below method will be used to get the complex type key array
-   *
-   * @return complex type key array
-   */
-  @Override public byte[][] getComplexTypeKeyArray() {
-    return getComplexTypeKeyArray(rowMapping[currentRow]);
-  }
-
-  /**
-   * Below method will be used to get the no dictionary key
-   * array for all the no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  @Override public byte[][] getNoDictionaryKeyArray() {
-    return getNoDictionaryKeyArray(rowMapping[currentRow]);
-  }
-
-  /**
-   * will return the current valid row id
-   *
-   * @return valid row id
-   */
-  @Override public int getCurrenrRowId() {
-    return rowMapping[currentRow];
-  }
-
-  /**
-   * Return the dimension data based on dimension ordinal
-   *
-   * @param dimensionOrdinal dimension ordinal
-   * @return dimension data
-   */
-  @Override public byte[] getDimensionKey(int dimensionOrdinal) {
-    return getDimensionData(dimensionOrdinal, rowMapping[currentRow]);
-  }
-
-  /**
-   * Below method will be used to to check whether measure value
-   * is null or for a measure
-   *
-   * @param ordinal measure ordinal
-   * @return is null or not
-   */
-  @Override public boolean isNullMeasureValue(int ordinal) {
-    return isNullMeasureValue(ordinal, rowMapping[currentRow]);
-  }
-
-  /**
-   * Below method will be used to get the measure value for measure
-   * of long data type
-   *
-   * @param ordinal measure ordinal
-   * @return long value of measure
-   */
-  @Override public long getLongMeasureValue(int ordinal) {
-    return getLongMeasureValue(ordinal, rowMapping[currentRow]);
-  }
-
-  /**
-   * Below method will be used to get the value of measure of double
-   * type
-   *
-   * @param ordinal measure ordinal
-   * @return measure value
-   */
-  @Override public double getDoubleMeasureValue(int ordinal) {
-    return getDoubleMeasureValue(ordinal, rowMapping[currentRow]);
-  }
-
-  /**
-   * Below method will be used to get the data of big decimal type
-   * of a measure
-   *
-   * @param ordinal measure ordinal
-   * @return measure value
-   */
-  @Override public BigDecimal getBigDecimalMeasureValue(int ordinal) {
-    return getBigDecimalMeasureValue(ordinal, rowMapping[currentRow]);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
deleted file mode 100644
index 3f31028..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/impl/ListBasedResult.java
+++ /dev/null
@@ -1,153 +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.carbondata.query.carbon.result.impl;
-
-import java.util.ArrayList;
-import java.util.List;
-
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.wrappers.ByteArrayWrapper;
-
-/**
- * Below class is a holder over list based result wrapper
- */
-public class ListBasedResult implements Result<List<ListBasedResultWrapper>, Object> {
-
-  /**
-   * current result list
-   */
-  private List<ListBasedResultWrapper> currentRowPointer;
-
-  /**
-   * all result list , this is required because if we merger all the scanned
-   * result from all the blocks in one list, that list creation will take more
-   * time as every time list will create a big array and then it will do copy
-   * the older element to new array, and creation of big array will also be a
-   * problem if memory is fragmented then jvm in to do defragmentation to
-   * create a big space, but if divide the data in multiple list than it avoid
-   * copy and defragmentation
-   */
-  private List<List<ListBasedResultWrapper>> allRowsResult;
-
-  /**
-   * counter to check how many result processed
-   */
-  private int totalRecordCounter = -1;
-
-  /**
-   * number of records
-   */
-  private int totalNumberOfRecords;
-
-  /**
-   * current counter of the record in list
-   */
-  private int listRecordCounter = -1;
-
-  /**
-   * current list counter
-   */
-  private int currentListCounter;
-
-  public ListBasedResult() {
-    currentRowPointer =
-        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-    allRowsResult =
-        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
-  }
-
-  /**
-   * below method will be used to add the scan result
-   */
-  @Override public void addScannedResult(List<ListBasedResultWrapper> listBasedResult) {
-    this.currentRowPointer = listBasedResult;
-    totalNumberOfRecords = listBasedResult.size();
-    allRowsResult.add(listBasedResult);
-  }
-
-  /**
-   * Method to check more result is present
-   * or not
-   */
-  @Override public boolean hasNext() {
-    if (allRowsResult.size() == 0) {
-      return false;
-    }
-    // As we are storing data in list of list, below code is to check whether
-    // any more result is present
-    // in the result.
-    // first it will check list counter is zero if it is zero
-    // than it will check list counter to check how many list has been processed
-    // if more list are present and all the list of current list is processed
-    // than it will take a new list from all row result list
-    totalRecordCounter++;
-    listRecordCounter++;
-    if (listRecordCounter == 0 || (listRecordCounter >= currentRowPointer.size()
-        && currentListCounter < allRowsResult.size())) {
-      listRecordCounter = 0;
-      currentRowPointer = allRowsResult.get(currentListCounter);
-      currentListCounter++;
-    }
-    return totalRecordCounter < totalNumberOfRecords;
-  }
-
-  /**
-   * @return key
-   */
-  @Override public ByteArrayWrapper getKey() {
-    return currentRowPointer.get(listRecordCounter).getKey();
-  }
-
-  /**
-   * @return will return the value
-   */
-  @Override public Object[] getValue() {
-    return currentRowPointer.get(listRecordCounter).getValue();
-  }
-
-  /***
-   * below method will be used to merge the
-   * scanned result
-   *
-   * @param otherResult return to be merged
-   */
-  @Override public void merge(Result<List<ListBasedResultWrapper>, Object> otherResult) {
-    if (otherResult.size() > 0) {
-      totalNumberOfRecords += otherResult.size();
-      this.allRowsResult.add(otherResult.getResult());
-    }
-  }
-
-  /**
-   * Return the size of the result
-   */
-  @Override public int size() {
-    return totalNumberOfRecords;
-  }
-
-  /**
-   * @return the complete result
-   */
-  @Override public List<ListBasedResultWrapper> getResult() {
-    return currentRowPointer;
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/impl/NonFilterQueryScannedResult.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/impl/NonFilterQueryScannedResult.java b/core/src/main/java/org/carbondata/query/carbon/result/impl/NonFilterQueryScannedResult.java
deleted file mode 100644
index 1268575..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/impl/NonFilterQueryScannedResult.java
+++ /dev/null
@@ -1,109 +0,0 @@
-package org.carbondata.query.carbon.result.impl;
-
-import java.math.BigDecimal;
-
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.result.AbstractScannedResult;
-
-/**
- * Result provide class for non filter query
- * In case of no filter query we need to return
- * complete data
- */
-public class NonFilterQueryScannedResult extends AbstractScannedResult {
-
-  public NonFilterQueryScannedResult(BlockExecutionInfo blockExecutionInfo) {
-    super(blockExecutionInfo);
-  }
-
-  /**
-   * @return dictionary key array for all the dictionary dimension selected in
-   * query
-   */
-  @Override public byte[] getDictionaryKeyArray() {
-    ++currentRow;
-    return getDictionaryKeyArray(currentRow);
-  }
-
-  /**
-   * Below method will be used to get the complex type key array
-   *
-   * @return complex type key array
-   */
-  @Override public byte[][] getComplexTypeKeyArray() {
-    return getComplexTypeKeyArray(currentRow);
-  }
-
-  /**
-   * Below method will be used to get the no dictionary key array for all the
-   * no dictionary dimension selected in query
-   *
-   * @return no dictionary key array for all the no dictionary dimension
-   */
-  @Override public byte[][] getNoDictionaryKeyArray() {
-    return getNoDictionaryKeyArray(currentRow);
-  }
-
-  /**
-   * will return the current valid row id
-   *
-   * @return valid row id
-   */
-  @Override public int getCurrenrRowId() {
-    return currentRow;
-  }
-
-  /**
-   * Return the dimension data based on dimension ordinal
-   *
-   * @param dimensionOrdinal dimension ordinal
-   * @return dimension data
-   */
-  @Override public byte[] getDimensionKey(int dimensionOrdinal) {
-    return getDimensionData(dimensionOrdinal, currentRow);
-  }
-
-  /**
-   * Below method will be used to to check whether measure value is null or
-   * for a measure
-   *
-   * @param ordinal measure ordinal
-   * @return is null or not
-   */
-  @Override public boolean isNullMeasureValue(int ordinal) {
-    return isNullMeasureValue(ordinal, currentRow);
-  }
-
-  /**
-   * Below method will be used to get the measure value for measure of long
-   * data type
-   *
-   * @param ordinal measure ordinal
-   * @return long value of measure
-   */
-  @Override public long getLongMeasureValue(int ordinal) {
-    return getLongMeasureValue(ordinal, currentRow);
-  }
-
-  /**
-   * Below method will be used to get the value of measure of double type
-   *
-   * @param ordinal measure ordinal
-   * @return measure value
-   */
-  @Override public double getDoubleMeasureValue(int ordinal) {
-    return getDoubleMeasureValue(ordinal, currentRow);
-  }
-
-  /**
-   * Below method will be used to get the data of big decimal type of a
-   * measure
-   *
-   * @param ordinal measure ordinal
-   * @return measure value
-   */
-  @Override public BigDecimal getBigDecimalMeasureValue(int ordinal) {
-    return getBigDecimalMeasureValue(ordinal, currentRow);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
deleted file mode 100644
index d2aa014..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/AbstractDetailQueryResultIterator.java
+++ /dev/null
@@ -1,134 +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.carbondata.query.carbon.result.iterator;
-
-import java.util.List;
-
-import org.carbondata.common.logging.LogService;
-import org.carbondata.common.logging.LogServiceFactory;
-import org.carbondata.core.carbon.datastore.DataRefNode;
-import org.carbondata.core.carbon.datastore.DataRefNodeFinder;
-import org.carbondata.core.carbon.datastore.impl.btree.BTreeDataRefNodeFinder;
-import org.carbondata.core.constants.CarbonCommonConstants;
-import org.carbondata.core.datastorage.store.FileHolder;
-import org.carbondata.core.datastorage.store.impl.FileFactory;
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.core.util.CarbonProperties;
-import org.carbondata.query.carbon.executor.infos.BlockExecutionInfo;
-import org.carbondata.query.carbon.executor.internal.InternalQueryExecutor;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.processor.AbstractDataBlockIterator;
-import org.carbondata.query.carbon.processor.impl.DataBlockIteratorImpl;
-
-/**
- * In case of detail query we cannot keep all the records in memory so for
- * executing that query are returning a iterator over block and every time next
- * call will come it will execute the block and return the result
- */
-public abstract class AbstractDetailQueryResultIterator extends CarbonIterator {
-
-  /**
-   * LOGGER.
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(AbstractDetailQueryResultIterator.class.getName());
-
-  /**
-   * execution info of the block
-   */
-  protected List<BlockExecutionInfo> blockExecutionInfos;
-
-  /**
-   * executor which will execute the query
-   */
-  protected InternalQueryExecutor executor;
-
-  /**
-   * number of cores which can be used
-   */
-  private int batchSize;
-
-  /**
-   * file reader which will be used to execute the query
-   */
-  protected FileHolder fileReader;
-
-  protected AbstractDataBlockIterator dataBlockIterator;
-
-  protected boolean nextBatch = false;
-
-  public AbstractDetailQueryResultIterator(List<BlockExecutionInfo> infos, QueryModel queryModel) {
-    String batchSizeString =
-        CarbonProperties.getInstance().getProperty(CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE);
-    if (null != batchSizeString) {
-      try {
-        batchSize = Integer.parseInt(batchSizeString);
-      } catch (NumberFormatException ne) {
-        LOGGER.error("Invalid inmemory records size. Using default value");
-        batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
-      }
-    } else {
-      batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
-    }
-
-    this.blockExecutionInfos = infos;
-    this.fileReader = FileFactory.getFileHolder(
-        FileFactory.getFileType(queryModel.getAbsoluteTableIdentifier().getStorePath()));
-    intialiseInfos();
-  }
-
-  private void intialiseInfos() {
-    for (BlockExecutionInfo blockInfo : blockExecutionInfos) {
-      DataRefNodeFinder finder = new BTreeDataRefNodeFinder(blockInfo.getEachColumnValueSize());
-      DataRefNode startDataBlock = finder
-          .findFirstDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getStartKey());
-      DataRefNode endDataBlock = finder
-          .findLastDataBlock(blockInfo.getDataBlock().getDataRefNode(), blockInfo.getEndKey());
-      long numberOfBlockToScan = endDataBlock.nodeNumber() - startDataBlock.nodeNumber() + 1;
-      blockInfo.setFirstDataBlock(startDataBlock);
-      blockInfo.setNumberOfBlockToScan(numberOfBlockToScan);
-    }
-  }
-
-  @Override public boolean hasNext() {
-    if ((dataBlockIterator != null && dataBlockIterator.hasNext()) || nextBatch) {
-      return true;
-    } else {
-      dataBlockIterator = getDataBlockIterator();
-      while (dataBlockIterator != null) {
-        if (dataBlockIterator.hasNext()) {
-          return true;
-        }
-        dataBlockIterator = getDataBlockIterator();
-      }
-      return false;
-    }
-  }
-
-  private DataBlockIteratorImpl getDataBlockIterator() {
-    if(blockExecutionInfos.size() > 0) {
-      BlockExecutionInfo executionInfo = blockExecutionInfos.get(0);
-      blockExecutionInfos.remove(executionInfo);
-      return new DataBlockIteratorImpl(executionInfo, fileReader, batchSize);
-    }
-    return null;
-  }
-
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java
deleted file mode 100644
index 826f816..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkBasedDetailResultIterator.java
+++ /dev/null
@@ -1,75 +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.carbondata.query.carbon.result.iterator;
-
-import java.util.List;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.executor.impl.QueryExecutorProperties;
-import org.carbondata.query.carbon.model.QueryModel;
-import org.carbondata.query.carbon.result.BatchResult;
-import org.carbondata.query.carbon.result.ListBasedResultWrapper;
-import org.carbondata.query.carbon.result.Result;
-import org.carbondata.query.carbon.result.preparator.QueryResultPreparator;
-import org.carbondata.query.carbon.result.preparator.impl.DetailQueryResultPreparatorImpl;
-
-/**
- * Iterator over chunk result
- */
-public class ChunkBasedDetailResultIterator extends CarbonIterator<BatchResult> {
-
-  /**
-   * query result prepartor which will be used to create a query result
-   */
-  private QueryResultPreparator<List<ListBasedResultWrapper>, Object> queryResultPreparator;
-
-  /**
-   * iterator over result
-   */
-  private CarbonIterator<Result> queryResultIterator;
-
-  public ChunkBasedDetailResultIterator(CarbonIterator<Result> queryResultIterator,
-      QueryExecutorProperties executerProperties, QueryModel queryModel) {
-    this.queryResultIterator = queryResultIterator;
-    this.queryResultPreparator =
-        new DetailQueryResultPreparatorImpl(executerProperties, queryModel);
-
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements. (In other words,
-   * returns {@code true}
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    return queryResultIterator.hasNext();
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public BatchResult next() {
-    return queryResultPreparator.prepareQueryResult(queryResultIterator.next());
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/1c725f5b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java b/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java
deleted file mode 100644
index 3db3404..0000000
--- a/core/src/main/java/org/carbondata/query/carbon/result/iterator/ChunkRowIterator.java
+++ /dev/null
@@ -1,79 +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.carbondata.query.carbon.result.iterator;
-
-import org.carbondata.core.iterator.CarbonIterator;
-import org.carbondata.query.carbon.result.BatchResult;
-
-/**
- * Iterator over row result
- */
-public class ChunkRowIterator extends CarbonIterator<Object[]> {
-
-  /**
-   * iterator over chunk result
-   */
-  private CarbonIterator<BatchResult> iterator;
-
-  /**
-   * currect chunk
-   */
-  private BatchResult currentchunk;
-
-  public ChunkRowIterator(CarbonIterator<BatchResult> iterator) {
-    this.iterator = iterator;
-    if (iterator.hasNext()) {
-      currentchunk = iterator.next();
-    }
-  }
-
-  /**
-   * Returns {@code true} if the iteration has more elements. (In other words,
-   * returns {@code true} if {@link #next} would return an element rather than
-   * throwing an exception.)
-   *
-   * @return {@code true} if the iteration has more elements
-   */
-  @Override public boolean hasNext() {
-    if (null != currentchunk) {
-      if ((currentchunk.hasNext())) {
-        return true;
-      } else if (!currentchunk.hasNext()) {
-        while (iterator.hasNext()) {
-          currentchunk = iterator.next();
-          if (currentchunk != null && currentchunk.hasNext()) {
-            return true;
-          }
-        }
-      }
-    }
-    return false;
-  }
-
-  /**
-   * Returns the next element in the iteration.
-   *
-   * @return the next element in the iteration
-   */
-  @Override public Object[] next() {
-    return currentchunk.next();
-  }
-
-}