You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/06/30 17:41:48 UTC

[01/50] [abbrv] incubator-carbondata git commit: [BUG] Failed to update table status on concurrent compaction and data load. (#741)

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master e9bbf75c2 -> 6e943ff73


[BUG] Failed to update table status on concurrent compaction and data load. (#741)

* If table status lock is failed, retry to acquire, then complete data load or the compaction should fail.

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

Branch: refs/heads/master
Commit: b0f2f060f82669a4c62b2b7d6d649e49243846ba
Parents: 841e59d
Author: ravikiran23 <ra...@gmail.com>
Authored: Sat Jun 25 00:49:58 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Sat Jun 25 00:49:58 2016 +0530

----------------------------------------------------------------------
 .../carbondata/core/locks/LocalFileLock.java    |   4 +-
 .../org/carbondata/core/locks/LockUsage.java    |   3 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java | 104 +++++++++++------
 .../spark/merger/CarbonDataMergerUtil.java      | 102 ++++++++++-------
 .../spark/rdd/CarbonDataRDDFactory.scala        |  11 +-
 .../datacompaction/DataCompactionLockTest.scala | 111 +++++++++++++++++++
 .../lcm/status/SegmentStatusManager.java        |  13 +++
 7 files changed, 269 insertions(+), 79 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b0f2f060/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
index 362b86f..2b26106 100644
--- a/core/src/main/java/org/carbondata/core/locks/LocalFileLock.java
+++ b/core/src/main/java/org/carbondata/core/locks/LocalFileLock.java
@@ -84,9 +84,9 @@ public class LocalFileLock extends AbstractCarbonLock {
     this.lockUsage = lockUsage;
     location = location.replace("\\", "/");
     String tempStr = location.substring(0, location.lastIndexOf('/'));
+    cubeName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
+    tempStr = tempStr.substring(0, tempStr.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;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b0f2f060/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
index 850b6bf..f07cfb8 100644
--- a/core/src/main/java/org/carbondata/core/locks/LockUsage.java
+++ b/core/src/main/java/org/carbondata/core/locks/LockUsage.java
@@ -24,6 +24,7 @@ package org.carbondata.core.locks;
  */
 public enum LockUsage {
   METADATA_LOCK,
-  COMPACTION_LOCK;
+  COMPACTION_LOCK,
+  TABLE_STATUS_LOCK;
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b0f2f060/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 f78c328..427c7e5 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
@@ -29,6 +29,7 @@ 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.CarbonDataLoadSchema;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
@@ -48,6 +49,7 @@ 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.load.LoadMetadataDetails;
+import org.carbondata.core.locks.ICarbonLock;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
@@ -692,52 +694,82 @@ public final class CarbonLoaderUtil {
   /**
    * This API will write the load level metadata for the loadmanagement module inorder to
    * manage the load and query execution management smoothly.
+   *
+   * @param loadCount
+   * @param loadMetadataDetails
+   * @param loadModel
+   * @param loadStatus
+   * @param startLoadTime
+   * @return boolean which determines whether status update is done or not.
+   * @throws IOException
    */
-  public static void recordLoadMetadata(int loadCount, LoadMetadataDetails loadMetadataDetails,
+  public static boolean recordLoadMetadata(int loadCount, LoadMetadataDetails loadMetadataDetails,
       CarbonLoadModel loadModel, String loadStatus, String startLoadTime) throws IOException {
 
-    String dataLoadLocation = null;
-    //String dataLoadLocation = getLoadFolderPath(loadModel);
-    dataLoadLocation =
-        loadModel.getCarbonDataLoadSchema().getCarbonTable().getMetaDataFilepath() + File.separator
-            + CarbonCommonConstants.LOADMETADATA_FILENAME;
-    Gson gsonObjectToRead = new Gson();
-    List<LoadMetadataDetails> listOfLoadFolderDetails = null;
-    DataInputStream dataInputStream = null;
-    String loadEnddate = readCurrentTime();
-    loadMetadataDetails.setTimestamp(loadEnddate);
-    loadMetadataDetails.setLoadStatus(loadStatus);
-    loadMetadataDetails.setLoadName(String.valueOf(loadCount));
-    loadMetadataDetails.setLoadStartTime(startLoadTime);
-    LoadMetadataDetails[] listOfLoadFolderDetailsArray = null;
-    try {
-      if (FileFactory.isFileExist(dataLoadLocation, FileFactory.getFileType(dataLoadLocation))) {
+    boolean status = false;
 
-        dataInputStream = FileFactory
-            .getDataInputStream(dataLoadLocation, FileFactory.getFileType(dataLoadLocation));
+    String metaDataFilepath =
+        loadModel.getCarbonDataLoadSchema().getCarbonTable().getMetaDataFilepath();
 
-        BufferedReader buffReader = new BufferedReader(new InputStreamReader(dataInputStream,
-            CarbonCommonConstants.CARBON_DEFAULT_STREAM_ENCODEFORMAT));
-        listOfLoadFolderDetailsArray =
-            gsonObjectToRead.fromJson(buffReader, LoadMetadataDetails[].class);
-      }
-      listOfLoadFolderDetails =
-          new ArrayList<LoadMetadataDetails>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    AbsoluteTableIdentifier absoluteTableIdentifier =
+        loadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
+
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            absoluteTableIdentifier.getCarbonTableIdentifier());
+
+    String tableStatusPath = carbonTablePath.getTableStatusFilePath();
+
+    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(
+        absoluteTableIdentifier);
+
+    ICarbonLock carbonLock = segmentStatusManager.getTableStatusLock();
 
-      if (null != listOfLoadFolderDetailsArray) {
-        for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
-          listOfLoadFolderDetails.add(loadMetadata);
+    try {
+      if (carbonLock.lockWithRetries()) {
+        LOGGER.info(
+            "Acquired lock for table" + loadModel.getDatabaseName() + "." + loadModel.getTableName()
+                + " for table status updation");
+
+        LoadMetadataDetails[] listOfLoadFolderDetailsArray =
+            segmentStatusManager.readLoadMetadata(metaDataFilepath);
+
+        String loadEnddate = readCurrentTime();
+        loadMetadataDetails.setTimestamp(loadEnddate);
+        loadMetadataDetails.setLoadStatus(loadStatus);
+        loadMetadataDetails.setLoadName(String.valueOf(loadCount));
+        loadMetadataDetails.setLoadStartTime(startLoadTime);
+
+        List<LoadMetadataDetails> listOfLoadFolderDetails =
+            new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+        if (null != listOfLoadFolderDetailsArray) {
+          for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
+            listOfLoadFolderDetails.add(loadMetadata);
+          }
         }
-      }
-      listOfLoadFolderDetails.add(loadMetadataDetails);
+        listOfLoadFolderDetails.add(loadMetadataDetails);
 
-    } finally {
+        segmentStatusManager.writeLoadDetailsIntoFile(tableStatusPath, listOfLoadFolderDetails
+            .toArray(new LoadMetadataDetails[listOfLoadFolderDetails.size()]));
 
-      CarbonUtil.closeStreams(dataInputStream);
+        status = true;
+      } else {
+        LOGGER.error("Not able to acquire the lock for Table status updation for table " + loadModel
+            .getDatabaseName() + "." + loadModel.getTableName());
+      }
+    } finally {
+      if (carbonLock.unlock()) {
+        LOGGER.info(
+            "Table unlocked successfully after table status updation" + loadModel.getDatabaseName()
+                + "." + loadModel.getTableName());
+      } else {
+        LOGGER.error(
+            "Unable to unlock Table lock for table" + loadModel.getDatabaseName() + "." + loadModel
+                .getTableName() + " during table status updation");
+      }
     }
-    writeLoadMetadata(loadModel.getCarbonDataLoadSchema(), loadModel.getDatabaseName(),
-        loadModel.getTableName(), listOfLoadFolderDetails);
-
+    return status;
   }
 
   public static void writeLoadMetadata(CarbonDataLoadSchema schema,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b0f2f060/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
index e26989c..f536293 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
@@ -44,6 +44,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.load.LoadMetadataDetails;
+import org.carbondata.core.locks.ICarbonLock;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.integration.spark.merger.CompactionType;
 import org.carbondata.lcm.status.SegmentStatusManager;
@@ -139,60 +140,85 @@ public final class CarbonDataMergerUtil {
     return CarbonCommonConstants.LOAD_FOLDER + segmentNumber;
   }
 
-  public static void updateLoadMetadataWithMergeStatus(List<LoadMetadataDetails> loadsToMerge,
+  public static boolean updateLoadMetadataWithMergeStatus(List<LoadMetadataDetails> loadsToMerge,
       String metaDataFilepath, String MergedLoadName, CarbonLoadModel carbonLoadModel,
       String mergeLoadStartTime) {
 
+    boolean tableStatusUpdationStatus = false;
     AbsoluteTableIdentifier absoluteTableIdentifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getAbsoluteTableIdentifier();
 
-    SegmentStatusManager segmentStatusManager = new SegmentStatusManager(absoluteTableIdentifier);
+    SegmentStatusManager segmentStatusManager =
+        new SegmentStatusManager(absoluteTableIdentifier);
 
-    CarbonTablePath carbonTablePath = CarbonStorePath
-        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
-            absoluteTableIdentifier.getCarbonTableIdentifier());
+    ICarbonLock carbonLock =
+        segmentStatusManager.getTableStatusLock();
 
-    String statusFilePath = carbonTablePath.getTableStatusFilePath();
+    try {
+      if (carbonLock.lockWithRetries()) {
+        LOGGER.info("Acquired lock for the table " + carbonLoadModel.getDatabaseName() + "."
+            + carbonLoadModel.getTableName() + " for table status updation ");
 
-    LoadMetadataDetails[] loadDetails = segmentStatusManager.readLoadMetadata(metaDataFilepath);
+        CarbonTablePath carbonTablePath = CarbonStorePath
+            .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+                absoluteTableIdentifier.getCarbonTableIdentifier());
 
-    String mergedLoadNumber = MergedLoadName.substring(
-        MergedLoadName.lastIndexOf(CarbonCommonConstants.LOAD_FOLDER)
-            + CarbonCommonConstants.LOAD_FOLDER.length(), MergedLoadName.length());
+        String statusFilePath = carbonTablePath.getTableStatusFilePath();
 
-    String modificationOrDeletionTimeStamp = CarbonLoaderUtil.readCurrentTime();
-    for (LoadMetadataDetails loadDetail : loadDetails) {
-      // check if this segment is merged.
-      if (loadsToMerge.contains(loadDetail)) {
-        loadDetail.setLoadStatus(CarbonCommonConstants.SEGMENT_COMPACTED);
-        loadDetail.setModificationOrdeletionTimesStamp(modificationOrDeletionTimeStamp);
-        loadDetail.setMergedLoadName(mergedLoadNumber);
-      }
-    }
+        LoadMetadataDetails[] loadDetails = segmentStatusManager.readLoadMetadata(metaDataFilepath);
 
-    // create entry for merged one.
-    LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
-    loadMetadataDetails.setPartitionCount(carbonLoadModel.getPartitionId());
-    loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS);
-    String loadEnddate = CarbonLoaderUtil.readCurrentTime();
-    loadMetadataDetails.setTimestamp(loadEnddate);
-    loadMetadataDetails.setLoadName(mergedLoadNumber);
-    loadMetadataDetails.setLoadStartTime(mergeLoadStartTime);
-    loadMetadataDetails.setPartitionCount("0");
+        String mergedLoadNumber = MergedLoadName.substring(
+            MergedLoadName.lastIndexOf(CarbonCommonConstants.LOAD_FOLDER)
+                + CarbonCommonConstants.LOAD_FOLDER.length(), MergedLoadName.length());
 
-    List<LoadMetadataDetails> updatedDetailsList =
-        new ArrayList<LoadMetadataDetails>(Arrays.asList(loadDetails));
+        String modificationOrDeletionTimeStamp = CarbonLoaderUtil.readCurrentTime();
+        for (LoadMetadataDetails loadDetail : loadDetails) {
+          // check if this segment is merged.
+          if (loadsToMerge.contains(loadDetail)) {
+            loadDetail.setLoadStatus(CarbonCommonConstants.SEGMENT_COMPACTED);
+            loadDetail.setModificationOrdeletionTimesStamp(modificationOrDeletionTimeStamp);
+            loadDetail.setMergedLoadName(mergedLoadNumber);
+          }
+        }
 
-    // put the merged folder entry
-    updatedDetailsList.add(loadMetadataDetails);
+        // create entry for merged one.
+        LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
+        loadMetadataDetails.setPartitionCount(carbonLoadModel.getPartitionId());
+        loadMetadataDetails.setLoadStatus(CarbonCommonConstants.STORE_LOADSTATUS_SUCCESS);
+        String loadEnddate = CarbonLoaderUtil.readCurrentTime();
+        loadMetadataDetails.setTimestamp(loadEnddate);
+        loadMetadataDetails.setLoadName(mergedLoadNumber);
+        loadMetadataDetails.setLoadStartTime(mergeLoadStartTime);
+        loadMetadataDetails.setPartitionCount("0");
 
-    try {
-      segmentStatusManager.writeLoadDetailsIntoFile(statusFilePath,
-          updatedDetailsList.toArray(new LoadMetadataDetails[updatedDetailsList.size()]));
-    } catch (IOException e) {
-      LOGGER.error("Error while writing metadata");
-    }
+        List<LoadMetadataDetails> updatedDetailsList = new ArrayList<>(Arrays.asList(loadDetails));
 
+        // put the merged folder entry
+        updatedDetailsList.add(loadMetadataDetails);
+
+        try {
+          segmentStatusManager.writeLoadDetailsIntoFile(statusFilePath,
+              updatedDetailsList.toArray(new LoadMetadataDetails[updatedDetailsList.size()]));
+          tableStatusUpdationStatus = true;
+        } catch (IOException e) {
+          LOGGER.error("Error while writing metadata");
+        }
+      } else {
+        LOGGER.error(
+            "Could not able to obtain lock for table" + carbonLoadModel.getDatabaseName() + "."
+                + carbonLoadModel.getTableName() + "for table status updation");
+      }
+    } finally {
+      if (carbonLock.unlock()) {
+        LOGGER.info("Table unlocked successfully after table status updation" + carbonLoadModel
+            .getDatabaseName() + "." + carbonLoadModel.getTableName());
+      } else {
+        LOGGER.error(
+            "Unable to unlock Table lock for table" + carbonLoadModel.getDatabaseName() + "."
+                + carbonLoadModel.getTableName() + " during table status updation");
+      }
+    }
+    return tableStatusUpdationStatus;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b0f2f060/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 fc8d309..2575f0d 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
@@ -785,12 +785,19 @@ object CarbonDataRDDFactory extends Logging {
       } else {
         val metadataDetails = status(0)._2
         if (!isAgg) {
-          CarbonLoaderUtil
+          val status = CarbonLoaderUtil
             .recordLoadMetadata(currentLoadCount,
               metadataDetails,
               carbonLoadModel,
               loadStatus,
-              loadStartTime)
+              loadStartTime
+            )
+          if (!status) {
+            val message = "Dataload failed due to failure in table status updation."
+            logger.audit("Data load is failed.")
+            logger.error("Dataload failed due to failure in table status updation.")
+            throw new Exception(message)
+          }
         } else if (!carbonLoadModel.isRetentionRequest) {
           // TODO : Handle it
           logInfo("********Database updated**********")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b0f2f060/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
new file mode 100644
index 0000000..27fbb1a
--- /dev/null
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
@@ -0,0 +1,111 @@
+package org.carbondata.spark.testsuite.datacompaction
+
+import java.io.File
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.carbondata.core.carbon.path.{CarbonStorePath, CarbonTablePath}
+import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.carbondata.core.constants.CarbonCommonConstants
+import org.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
+import org.carbondata.core.util.CarbonProperties
+import org.carbondata.lcm.status.SegmentStatusManager
+import org.scalatest.BeforeAndAfterAll
+
+import scala.collection.JavaConverters._
+
+/**
+  * FT for data compaction Locking scenario.
+  */
+class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
+
+  val absoluteTableIdentifier: AbsoluteTableIdentifier = new
+      AbsoluteTableIdentifier(
+        CarbonProperties.getInstance.getProperty(CarbonCommonConstants.STORE_LOCATION),
+        new CarbonTableIdentifier("default", "compactionLockTestTable", "1")
+      )
+  val carbonTablePath: CarbonTablePath = CarbonStorePath
+    .getCarbonTablePath(absoluteTableIdentifier.getStorePath,
+      absoluteTableIdentifier.getCarbonTableIdentifier
+    )
+  val dataPath: String = carbonTablePath.getMetadataDirectoryPath
+
+  val carbonLock: ICarbonLock =
+    CarbonLockFactory.getCarbonLockObj(dataPath, LockUsage.TABLE_STATUS_LOCK)
+
+
+  override def beforeAll {
+    CarbonProperties.getInstance().addProperty("carbon.enable.load.merge", "true")
+    sql("drop table if exists  compactionLockTestTable")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "mm/dd/yyyy")
+    sql(
+      "CREATE TABLE IF NOT EXISTS compactionLockTestTable (country String, ID Int, date " +
+        "Timestamp, name " +
+        "String, " +
+        "phonetype String, serialname String, salary Int) STORED BY 'org.apache.carbondata" +
+        ".format'"
+    )
+
+    val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
+      .getCanonicalPath
+    var csvFilePath1 = currentDirectory + "/src/test/resources/compaction/compaction1.csv"
+
+    var csvFilePath2 = currentDirectory + "/src/test/resources/compaction/compaction2.csv"
+    var csvFilePath3 = currentDirectory + "/src/test/resources/compaction/compaction3.csv"
+
+    sql("LOAD DATA fact from '" + csvFilePath1 + "' INTO CUBE compactionLockTestTable " +
+      "PARTITIONDATA" +
+      "(DELIMITER ',', QUOTECHAR '\"')"
+    )
+    sql("LOAD DATA fact from '" + csvFilePath2 + "' INTO CUBE compactionLockTestTable  " +
+      "PARTITIONDATA" +
+      "(DELIMITER ',', QUOTECHAR '\"')"
+    )
+    sql("LOAD DATA fact from '" + csvFilePath3 + "' INTO CUBE compactionLockTestTable  " +
+      "PARTITIONDATA" +
+      "(DELIMITER ',', QUOTECHAR '\"')"
+    )
+    // take the lock so that next compaction will be failed.
+    carbonLock.lockWithRetries()
+
+    // compaction should happen here.
+    sql("alter table compactionLockTestTable compact 'major'"
+    )
+  }
+
+  /**
+    * Compaction should fail as lock is being held purposefully
+    */
+  test("check if compaction is failed or not.") {
+    var status = true
+    var noOfRetries = 0
+    while (status && noOfRetries < 10) {
+
+      val segmentStatusManager: SegmentStatusManager = new SegmentStatusManager(
+        absoluteTableIdentifier
+      )
+      val segments = segmentStatusManager.getValidSegments().listOfValidSegments.asScala.toList
+
+      if (!segments.contains("0.1")) {
+        // wait for 2 seconds for compaction to complete.
+        Thread.sleep(2000)
+        noOfRetries += 1
+      }
+      else {
+        status = false
+      }
+    }
+    assert(status)
+  }
+
+
+  override def afterAll {
+    /* sql("drop cube compactionLockTestTable") */
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+    carbonLock.unlock()
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b0f2f060/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..8e4c627 100644
--- a/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
@@ -70,6 +70,19 @@ public class SegmentStatusManager {
   }
 
   /**
+   * This will return the lock object used to lock the table status file before updation.
+   *
+   * @return
+   */
+  public ICarbonLock getTableStatusLock() {
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            absoluteTableIdentifier.getCarbonTableIdentifier());
+    String metaDataFilepath = carbonTablePath.getMetadataDirectoryPath();
+    return CarbonLockFactory.getCarbonLockObj(metaDataFilepath, LockUsage.TABLE_STATUS_LOCK);
+  }
+
+  /**
    * get valid segment for given table
    * @return
    * @throws IOException


[20/50] [abbrv] incubator-carbondata git commit: [issue-CARBONDATA-14] arbon.cutOffTimestamp and carbon.timegranularity is not added in the carbon.properties.template (#758)

Posted by ch...@apache.org.
[issue-CARBONDATA-14] arbon.cutOffTimestamp and carbon.timegranularity is not added in the carbon.properties.template (#758)

[issue-CARBONDATA-14] Carbon.cutOffTimestamp and carbon.timegranularit\u2026

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

Branch: refs/heads/master
Commit: 5f86244347d78ab5d87bd374d76cf14489b5908f
Parents: 5045d73
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Mon Jun 27 15:46:10 2016 +0530
Committer: Vimal-Das <vi...@gmail.com>
Committed: Mon Jun 27 03:16:10 2016 -0700

----------------------------------------------------------------------
 conf/carbon.properties.template | 6 +++++-
 1 file changed, 5 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5f862443/conf/carbon.properties.template
----------------------------------------------------------------------
diff --git a/conf/carbon.properties.template b/conf/carbon.properties.template
index 8dd6598..3d13fd7 100644
--- a/conf/carbon.properties.template
+++ b/conf/carbon.properties.template
@@ -91,4 +91,8 @@ carbon.enable.quick.filter=false
 ##threshold to identify whether high cardinality column
 #high.cardinality.threshold=1000000
 ##Percentage to identify whether column cardinality is more than configured percent of total row count
-#high.cardinality.row.count.percentage=80
\ No newline at end of file
+#high.cardinality.row.count.percentage=80
+##The property to set the date to be considered as start date for calculating the timestamp.
+#carbon.cutOffTimestamp=2000-01-01 00:00:00
+##The property to set the timestamp (ie milis) conversion to the SECOND, MINUTE, HOUR or DAY level.
+#carbon.timegranularity=SECOND
\ No newline at end of file


[38/50] [abbrv] incubator-carbondata git commit: [Bug] add more info to audit log (#781)

Posted by ch...@apache.org.
[Bug] add more info to audit log (#781)

Added use name, thread id, database name and table name to audit log.

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

Branch: refs/heads/master
Commit: fe7acdce243cb0e5d705d07a3627f4aa79338230
Parents: 5fdadba
Author: Venkata Ramana G <g....@gmail.com>
Authored: Thu Jun 30 04:52:15 2016 +0530
Committer: GitHub <no...@github.com>
Committed: Thu Jun 30 04:52:15 2016 +0530

----------------------------------------------------------------------
 common/pom.xml                                  |  8 ++-
 .../common/logging/impl/StandardLogService.java | 15 ++++-
 .../execution/command/carbonTableSchema.scala   | 58 ++++++++++----------
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |  6 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        | 19 ++++---
 5 files changed, 64 insertions(+), 42 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe7acdce/common/pom.xml
----------------------------------------------------------------------
diff --git a/common/pom.xml b/common/pom.xml
index fe4560e..d7b6ad3 100644
--- a/common/pom.xml
+++ b/common/pom.xml
@@ -55,6 +55,12 @@
         </exclusion>
       </exclusions>
     </dependency>
+    <dependency>
+      <groupId>org.apache.hadoop</groupId>
+      <artifactId>hadoop-common</artifactId>
+      <version>${hadoop.version}</version>
+      <scope>${hadoop.deps.scope}</scope>
+    </dependency>
   </dependencies>
   <build>
     <sourceDirectory>src/main/java</sourceDirectory>
@@ -69,4 +75,4 @@
       </plugin>
     </plugins>
   </build>
-</project>
\ No newline at end of file
+</project>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe7acdce/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java b/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
index c4c86bf..07cd7bb 100644
--- a/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
+++ b/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
@@ -19,12 +19,14 @@
 
 package org.carbondata.common.logging.impl;
 
+import java.io.IOException;
 import java.net.InetAddress;
 import java.net.UnknownHostException;
 import java.util.Properties;
 
 import org.carbondata.common.logging.LogService;
 
+import org.apache.hadoop.security.UserGroupInformation;
 import org.apache.log4j.Logger;
 import org.apache.log4j.MDC;
 
@@ -273,8 +275,17 @@ public final class StandardLogService implements LogService {
     } catch (UnknownHostException e) {
       hostName = "localhost";
     }
-
-    logger.log(AuditLevel.AUDIT, "[" + hostName + "]" + msg);
+    String username = "unknown";
+    String threadid = "unknown";
+    try {
+      threadid = Thread.currentThread().getId() + "";
+      username = UserGroupInformation.getCurrentUser().getShortUserName();
+    } catch (IOException e) {
+      username = "unknown";
+    }
+    logger.log(AuditLevel.AUDIT, "[" + hostName + "]"
+        + "[" + username + "]"
+        + "[Thread-" +  threadid + "]" + msg);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe7acdce/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 bcd6dd3..75abe0e 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
@@ -270,7 +270,7 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
       val name = f._1
       LOGGER.error(s"Duplicate column found with name : $name")
       LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation - " +
+        s"Validation failed for Create/Alter Table Operation for ${cm.schemaName}.${cm.cubeName}" +
         s"Duplicate column found with name : $name")
       sys.error(s"Duplicate dimensions found with name : $name")
     })
@@ -351,7 +351,8 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
           val msg = definedpartCols.mkString(", ")
           LOGGER.error(s"partition columns specified are not part of Dimension columns : $msg")
           LOGGER.audit(
-            s"Validation failed for Create/Alter Table Operation - " +
+            s"Validation failed for Create/Alter Table Operation for " +
+              s"${cm.schemaName}.${cm.cubeName} " +
             s"partition columns specified are not part of Dimension columns : $msg")
           sys.error(s"partition columns specified are not part of Dimension columns : $msg")
         }
@@ -363,7 +364,8 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
             case e: Exception =>
               val cl = part.partitionClass
               LOGGER.audit(
-                s"Validation failed for Create/Alter Table Operation - " +
+                s"Validation failed for Create/Alter Table Operation for " +
+                  s"${cm.schemaName}.${cm.cubeName} " +
                 s"partition class specified can not be found or loaded : $cl")
               sys.error(s"partition class specified can not be found or loaded : $cl")
           }
@@ -549,7 +551,7 @@ class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
       val name = f._1
       LOGGER.error(s"Duplicate dimensions found with name : $name")
       LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation - " +
+        s"Validation failed for Create/Alter Table Operation for ${cm.schemaName}.${cm.cubeName} " +
         s"Duplicate dimensions found with name : $name")
       sys.error(s"Duplicate dimensions found with name : $name")
     })
@@ -558,7 +560,7 @@ class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
       val name = f._1
       LOGGER.error(s"Duplicate dimensions found with column name : $name")
       LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation - " +
+        s"Validation failed for Create/Alter Table Operation for ${cm.schemaName}.${cm.cubeName} " +
         s"Duplicate dimensions found with column name : $name")
       sys.error(s"Duplicate dimensions found with column name : $name")
     })
@@ -567,7 +569,7 @@ class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
       val name = f._1
       LOGGER.error(s"Duplicate measures found with name : $name")
       LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation - " +
+        s"Validation failed for Create/Alter Table Operation for ${cm.schemaName}.${cm.cubeName} " +
         s"Duplicate measures found with name : $name")
       sys.error(s"Duplicate measures found with name : $name")
     })
@@ -576,7 +578,7 @@ class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
       val name = f._1
       LOGGER.error(s"Duplicate measures found with column name : $name")
       LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation - " +
+        s"Validation failed for Create/Alter Table Operation for ${cm.schemaName}.${cm.cubeName} " +
         s"Duplicate measures found with column name : $name")
       sys.error(s"Duplicate measures found with column name : $name")
     })
@@ -589,7 +591,8 @@ class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
         val fault = a.msrName
         LOGGER.error(s"Aggregator should not be defined for dimension fields [$fault]")
         LOGGER.audit(
-          s"Validation failed for Create/Alter Table Operation - " +
+          s"Validation failed for Create/Alter Table Operation for " +
+            s"${cm.schemaName}.${cm.cubeName} " +
           s"Aggregator should not be defined for dimension fields [$fault]")
         sys.error(s"Aggregator should not be defined for dimension fields [$fault]")
       }
@@ -599,7 +602,7 @@ class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
       val name = f._1
       LOGGER.error(s"Dimension and Measure defined with same name : $name")
       LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation - " +
+        s"Validation failed for Create/Alter Table Operation for ${cm.schemaName}.${cm.cubeName} " +
         s"Dimension and Measure defined with same name : $name")
       sys.error(s"Dimension and Measure defined with same name : $name")
     })
@@ -688,7 +691,8 @@ class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
             case e: Exception =>
               val cl = part.partitionClass
               LOGGER.audit(
-                s"Validation failed for Create/Alter Table Operation - " +
+                s"Validation failed for Create/Alter Table Operation for " +
+                  s"${cm.schemaName}.${cm.cubeName} " +
                 s"partition class specified can not be found or loaded : $cl")
               sys.error(s"partition class specified can not be found or loaded : $cl")
           }
@@ -1308,11 +1312,11 @@ private[sql] case class DeleteLoadsById(
 
   def run(sqlContext: SQLContext): Seq[Row] = {
 
-    LOGGER.audit("Delete load by Id request has been received.")
+    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    LOGGER.audit(s"Delete load by Id request has been received for $schemaName.$tableName")
 
     // validate load ids first
     validateLoadIds
-    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
       Option(schemaName),
@@ -1340,7 +1344,7 @@ private[sql] case class DeleteLoadsById(
 
     if (invalidLoadIds.isEmpty) {
 
-      LOGGER.audit("Delete load by Id is successfull.")
+      LOGGER.audit(s"Delete load by Id is successfull for $schemaName.$tableName.")
     }
     else {
       sys.error("Delete load by Id is failed. No matching load id found. SegmentSeqId(s) - "
@@ -1371,8 +1375,8 @@ private[sql] case class DeleteLoadsByLoadDate(
 
   def run(sqlContext: SQLContext): Seq[Row] = {
 
-    LOGGER.audit("Delete load by load date request has been received.")
     val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    LOGGER.audit(s"Delete load by load date request has been received for $schemaName.$tableName")
 
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
       Option(schemaName),
@@ -1408,7 +1412,7 @@ private[sql] case class DeleteLoadsByLoadDate(
     var invalidLoadTimestamps = segmentStatusManager
       .updateDeletionStatus(loadDate, path, timeObj.asInstanceOf[java.lang.Long]).asScala
     if(invalidLoadTimestamps.isEmpty) {
-      LOGGER.audit("Delete load by load date is successfull.")
+      LOGGER.audit(s"Delete load by load date is successfull for $schemaName.$tableName.")
     }
     else {
       sys.error("Delete load by load date is failed. No matching load found.")
@@ -1440,7 +1444,7 @@ 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)
+      LOGGER.audit(s"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")
@@ -1576,7 +1580,7 @@ private[sql] case class LoadCube(
       catch {
         case ex: Exception =>
           LOGGER.error(ex)
-          LOGGER.audit("Dataload failure. Please check the logs")
+          LOGGER.audit(s"Dataload failure for $schemaName.$tableName. Please check the logs")
           throw ex
       }
       finally {
@@ -1591,7 +1595,8 @@ private[sql] case class LoadCube(
         } catch {
           case ex: Exception =>
             LOGGER.error(ex)
-            LOGGER.audit("Dataload failure. Problem deleting the partition folder")
+            LOGGER.audit(s"Dataload failure for $schemaName.$tableName. " +
+              "Problem deleting the partition folder")
             throw ex
         }
 
@@ -1788,8 +1793,7 @@ private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Opti
     if (null == tmpTable) {
       if (!ifExistsSet) {
         LOGGER
-          .audit(s"Dropping carbon table with Database name [$schemaName] and Table name" +
-                 "[$cubeName] failed")
+          .audit(s"Dropping carbon table $schemaName.$cubeName failed")
         LOGGER.error(s"Carbon Table $schemaName.$cubeName metadata does not exist")
       }
       if (sqlContext.tableNames(schemaName).map(x => x.toLowerCase())
@@ -1814,9 +1818,7 @@ private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Opti
         if (carbonLock.lockWithRetries()) {
           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"failed as the Table is locked")
+          LOGGER.audit(s"Dropping table $schemaName.$cubeName failed as the Table is locked")
           sys.error("Table is locked for updation. Please try after some time")
         }
 
@@ -1840,7 +1842,7 @@ private[sql] case class DropCubeCommand(ifExistsSet: Boolean, schemaNameOp: Opti
               relation.cubeMeta.partitioner)
           QueryPartitionHelper.getInstance().removePartition(schemaName, cubeName)
 
-          LOGGER.audit(s"Deleted table [$cubeName] under database [$schemaName]")
+          LOGGER.audit(s"Deleted table $schemaName.$cubeName")
         }
       }
       finally {
@@ -2083,8 +2085,8 @@ 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)
+    LOGGER.audit(s"The delete load by date request has been received for $schemaName.$cubeName")
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
       .lookupRelation1(Some(schemaName), cubeName, None)(sqlContext).asInstanceOf[CarbonRelation]
     var level: String = ""
@@ -2102,7 +2104,7 @@ 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)
+        s"Table $schemaName.$cubeName does not contain date field :" + dateField)
       sys.error(s"Table $schemaName.$cubeName does not contain date field " + dateField)
     }
     else {
@@ -2123,7 +2125,7 @@ private[sql] case class DeleteLoadByDate(
       actualColName,
       dateValue,
       relation.cubeMeta.partitioner)
-    LOGGER.audit("The delete load by date is successfull.")
+    LOGGER.audit(s"The delete load by date $dateValue is successful for $schemaName.$cubeName.")
     Seq.empty
   }
 }
@@ -2136,8 +2138,8 @@ private[sql] case class CleanFiles(
   val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
 
   def run(sqlContext: SQLContext): Seq[Row] = {
-    LOGGER.audit("Clean files request has been received.")
     val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
+    LOGGER.audit(s"Clean files request has been received for $schemaName.$cubeName")
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
       .lookupRelation1(Some(schemaName), cubeName, None)(sqlContext).
       asInstanceOf[CarbonRelation]

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe7acdce/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 f1c8721..f88b74e 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
@@ -158,7 +158,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
           CarbonRelation(schemaName, cubeName,
             CarbonSparkUtil.createSparkMeta(cubes.head.carbonTable), cubes.head, alias)(sqlContext)
         } else {
-          LOGGER.audit(s"Table Not Found: $schemaName $cubeName")
+          LOGGER.audit(s"Table Not Found: $schemaName.$cubeName")
           throw new NoSuchTableException
         }
       case Seq(cubeName) =>
@@ -170,7 +170,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
           CarbonRelation(currentDatabase, cubeName,
             CarbonSparkUtil.createSparkMeta(cubes.head.carbonTable), cubes.head, alias)(sqlContext)
         } else {
-          LOGGER.audit(s"Table Not Found: $cubeName")
+          LOGGER.audit(s"Table Not Found: $currentDatabase.$cubeName")
           throw new NoSuchTableException
         }
       case _ =>
@@ -580,7 +580,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     val (timestampFile, timestampFileType) = getTimestampFileAndType(schemaName, cubeName)
 
     if (!FileFactory.isFileExist(timestampFile, timestampFileType)) {
-      LOGGER.audit("Creating timestamp file")
+      LOGGER.audit(s"Creating timestamp file for $schemaName.$cubeName")
       FileFactory.createNewFile(timestampFile, timestampFileType)
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/fe7acdce/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 6f44aef..1494282 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
@@ -213,7 +213,7 @@ object CarbonDataRDDFactory extends Logging {
       }
     } else {
       logError("Delete by Date request is failed")
-      logger.audit("The delete load by date is failed.")
+      logger.audit(s"The delete load by date is failed for $schemaName.$cubeName")
       sys.error("Delete by Date request is failed, potential causes " +
                 "Empty store or Invalid column type, For more details please refer logs.")
     }
@@ -305,8 +305,8 @@ object CarbonDataRDDFactory extends Logging {
     }
 
     logger
-      .audit("Compaction request received for table " + carbonLoadModel
-        .getDatabaseName + "." + carbonLoadModel.getTableName
+      .audit(s"Compaction request received for table " +
+        s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}"
       )
     val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
     val cubeCreationTime = CarbonEnv.getInstance(sqlContext).carbonCatalog
@@ -345,8 +345,8 @@ object CarbonDataRDDFactory extends Logging {
     }
     else {
       logger
-        .audit("Not able to acquire the compaction lock for table " + carbonLoadModel
-          .getDatabaseName + "." + carbonLoadModel.getTableName
+        .audit("Not able to acquire the compaction lock for table " +
+          s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}"
         )
       logger
         .error("Not able to acquire the compaction lock for table " + carbonLoadModel
@@ -827,7 +827,8 @@ object CarbonDataRDDFactory extends Logging {
           message = "Dataload failure"
         }
         logInfo("********clean up done**********")
-        logger.audit("Data load is failed.")
+        logger.audit(s"Data load is failed for " +
+          s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
         logWarning("Unable to write load metadata file")
         throw new Exception(message)
       } else {
@@ -842,7 +843,8 @@ object CarbonDataRDDFactory extends Logging {
             )
           if (!status) {
             val message = "Dataload failed due to failure in table status updation."
-            logger.audit("Data load is failed.")
+            logger.audit("Data load is failed for " +
+              s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
             logger.error("Dataload failed due to failure in table status updation.")
             throw new Exception(message)
           }
@@ -850,7 +852,8 @@ object CarbonDataRDDFactory extends Logging {
           // TODO : Handle it
           logInfo("********Database updated**********")
         }
-        logger.audit("Data load is successful.")
+        logger.audit("Data load is successful for " +
+          s"${carbonLoadModel.getDatabaseName}.${carbonLoadModel.getTableName}")
       }
     }
 


[47/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
index a7d98a3,0000000..d3423b0
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
+++ b/core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
@@@ -1,718 -1,0 +1,755 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 +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) {
++      List<String> dictionaryColumnIdList, CarbonTableIdentifier carbonTableIdentifier)
++      throws QueryExecutionException {
 +    CarbonTable carbonTable =
 +        CarbonMetadata.getInstance().getCarbonTable(carbonTableIdentifier.getTableUniqueName());
 +    List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
 +        new ArrayList<>(dictionaryColumnIdList.size());
-     for (String columnIdentifier : dictionaryColumnIdList) {
++    for (String columnId : dictionaryColumnIdList) {
 +      CarbonDimension dimension = CarbonMetadata.getInstance()
-           .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnIdentifier);
++          .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnId);
++      if (null == dimension) {
++        throw new QueryExecutionException(
++            "The column id " + columnId + " could not be resolved.");
++      }
 +      DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-           new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
-               dimension.getDataType());
++          new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
++              dimension.getColumnIdentifier(), 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();
++    Map<Integer, KeyGenerator> columnGroupAndItsKeygenartor =
++        segmentProperties.getColumnGroupAndItsKeygenartor();
 +
 +    Iterator<Entry<Integer, List<Integer>>> iterator =
 +        columnGroupAndItsOrdinalMappingForQuery.entrySet().iterator();
 +    KeyStructureInfo restructureInfos = null;
 +    while (iterator.hasNext()) {
 +      Entry<Integer, List<Integer>> next = iterator.next();
++      KeyGenerator keyGenerator = columnGroupAndItsKeygenartor.get(next.getKey());
 +      restructureInfos = new KeyStructureInfo();
 +      // sort the ordinal
 +      List<Integer> ordinal = next.getValue();
-       Collections.sort(ordinal);
++      List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
++      for (Integer ord : ordinal) {
++        mdKeyOrdinal.add(segmentProperties.getColumnGroupMdKeyOrdinal(next.getKey(), ord));
++      }
++      Collections.sort(mdKeyOrdinal);
 +      // get the masked byte range for column group
-       int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(ordinal, keyGenerator);
++      int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
 +      // max key for column group
-       byte[] maxKey = getMaxKeyBasedOnOrinal(ordinal, keyGenerator);
++      byte[] maxKey = getMaxKeyBasedOnOrinal(mdKeyOrdinal, 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) {
++          && dimensions.get(index).getDimension().columnGroupId() == prvColumnGroupId
++          && null != currentColumnGroup) {
 +        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
++      // first need to check any measure is present or not and as if measure
++      // if measure is present and if first measure is not a default
++      // measure than add measure otherwise
 +      // than add first dimension as a measure
++      //as currently if measure is not present then
++      //we are adding default measure so first condition will
++      //never come false but if in future we can remove so not removing first if check
 +      if (queryMeasure.getColumnName().equals("count(*)")) {
-         if (carbonTable.getMeasureByTableName(tableName).size() > 0) {
++        if (carbonTable.getMeasureByTableName(tableName).size() > 0 && !carbonTable
++            .getMeasureByTableName(tableName).get(0).getColName()
++            .equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)) {
 +          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()]));
 +  }
++
++  /**
++   * It is required for extracting column data from columngroup chunk
++   *
++   * @return
++   * @throws KeyGenException
++   */
++  public static KeyStructureInfo getKeyStructureInfo(SegmentProperties segmentProperties,
++      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo) throws KeyGenException {
++    int colGrpId = getColumnGroupId(segmentProperties, dimColumnEvaluatorInfo.getColumnIndex());
++    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
++    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
++
++    mdKeyOrdinal.add(segmentProperties
++        .getColumnGroupMdKeyOrdinal(colGrpId, dimColumnEvaluatorInfo.getColumnIndex()));
++    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
++    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
++    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
++    KeyStructureInfo restructureInfos = new KeyStructureInfo();
++    restructureInfos.setKeyGenerator(keyGenerator);
++    restructureInfos.setMaskByteRanges(maskByteRanges);
++    restructureInfos.setMaxKey(maxKey);
++    restructureInfos.setMaskedBytes(maksedByte);
++    return restructureInfos;
++  }
++
++  public static int getColumnGroupId(SegmentProperties segmentProperties, int ordinal) {
++    int[][] columnGroups = segmentProperties.getColumnGroups();
++    int colGrpId = -1;
++    for (int i = 0; i < columnGroups.length; i++) {
++      if (columnGroups[i].length > 1) {
++        colGrpId++;
++        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
++          break;
++        }
++      }
++    }
++    return colGrpId;
++  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java
index 922e706,0000000..3807d29
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/ColumnExpression.java
@@@ -1,112 -1,0 +1,113 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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));
++    ExpressionResult expressionResult =
++        new ExpressionResult(dataType, (null == value ? null : 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/7f722186/core/src/main/java/org/carbondata/scan/expression/Expression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/Expression.java
index 01b4fee,0000000..31af903
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/Expression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/Expression.java
@@@ -1,48 -1,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.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.FilterIllegalMemberException;
 +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 ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException;
 +
 +  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/7f722186/core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
index a86052c,0000000..63afabd
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
+++ b/core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
@@@ -1,466 -1,0 +1,474 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for 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;
++import org.carbondata.scan.expression.exception.FilterIllegalMemberException;
 +
 +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 {
++  public Integer getInt() throws FilterIllegalMemberException {
 +    if (value == null) {
 +      return null;
 +    }
 +    try {
 +      switch (this.getDataType()) {
 +        case StringType:
 +          try {
 +            return Integer.parseInt(value.toString());
 +          } catch (NumberFormatException e) {
-             throw new FilterUnsupportedException(e);
++            throw new FilterIllegalMemberException(e);
 +          }
 +
 +        case ShortType:
 +          return ((Short) value).intValue();
 +        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(
++          throw new FilterIllegalMemberException(
 +              "Cannot convert" + this.getDataType().name() + " to integer type value");
 +      }
 +
 +    } catch (ClassCastException e) {
-       throw new FilterUnsupportedException(
++      throw new FilterIllegalMemberException(
 +          "Cannot convert" + this.getDataType().name() + " to Integer type value");
 +    }
 +  }
 +
-   public Short getShort() throws FilterUnsupportedException {
++  public Short getShort() throws FilterIllegalMemberException {
 +    if (value == null) {
 +      return null;
 +    }
 +    try {
 +      switch (this.getDataType()) {
 +        case StringType:
 +          try {
 +            return Short.parseShort(value.toString());
 +          } catch (NumberFormatException e) {
-             throw new FilterUnsupportedException(e);
++            throw new FilterIllegalMemberException(e);
 +          }
 +        case ShortType:
 +        case IntegerType:
 +        case DoubleType:
 +
 +          if (value instanceof Double) {
 +            return ((Double) value).shortValue();
 +          } else if (value instanceof Integer) {
 +            return ((Integer) value).shortValue();
 +          }
 +          return (Short) value;
 +
 +        case TimestampType:
 +
 +          if (value instanceof Timestamp) {
 +            return (short) (((Timestamp) value).getTime() % 1000);
 +          } else {
 +            return (Short) value;
 +          }
 +
 +        default:
-           throw new FilterUnsupportedException(
++          throw new FilterIllegalMemberException(
 +              "Cannot convert" + this.getDataType().name() + " to integer type value");
 +      }
 +
 +    } catch (ClassCastException e) {
-       throw new FilterUnsupportedException(
++      throw new FilterIllegalMemberException(
 +          "Cannot convert" + this.getDataType().name() + " to Integer type value");
 +    }
 +  }
 +
-   public String getString() {
++  public String getString() throws FilterIllegalMemberException {
 +    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();
++    try {
++      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();
++      }
++    } catch (Exception e) {
++      throw new FilterIllegalMemberException(
++          "Cannot convert" + this.getDataType().name() + " to String type value");
 +    }
 +  }
 +
-   public Double getDouble() throws FilterUnsupportedException {
++  public Double getDouble() throws FilterIllegalMemberException {
 +    if (value == null) {
 +      return null;
 +    }
 +    try {
 +      switch (this.getDataType()) {
 +        case StringType:
 +          try {
 +            return Double.parseDouble(value.toString());
 +          } catch (NumberFormatException e) {
-             throw new FilterUnsupportedException(e);
++            throw new FilterIllegalMemberException(e);
 +          }
 +        case ShortType:
 +          return ((Short) value).doubleValue();
 +        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(
++          throw new FilterIllegalMemberException(
 +              "Cannot convert" + this.getDataType().name() + " to double type value");
 +      }
 +    } catch (ClassCastException e) {
-       throw new FilterUnsupportedException(
++      throw new FilterIllegalMemberException(
 +          "Cannot convert" + this.getDataType().name() + " to Double type value");
 +    }
 +  }
 +  //CHECKSTYLE:ON
 +
-   public Long getLong() throws FilterUnsupportedException {
++  public Long getLong() throws FilterIllegalMemberException {
 +    if (value == null) {
 +      return null;
 +    }
 +    try {
 +      switch (this.getDataType()) {
 +        case StringType:
 +          try {
 +            return Long.parseLong(value.toString());
 +          } catch (NumberFormatException e) {
-             throw new FilterUnsupportedException(e);
++            throw new FilterIllegalMemberException(e);
 +          }
 +        case ShortType:
 +          return ((Short) value).longValue();
 +        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(
++          throw new FilterIllegalMemberException(
 +              "Cannot convert" + this.getDataType().name() + " to Long type value");
 +      }
 +    } catch (ClassCastException e) {
-       throw new FilterUnsupportedException(
++      throw new FilterIllegalMemberException(
 +          "Cannot convert" + this.getDataType().name() + " to Long type value");
 +    }
 +
 +  }
 +
 +  //Add to judge for BigDecimal
-   public BigDecimal getDecimal() throws FilterUnsupportedException {
++  public BigDecimal getDecimal() throws FilterIllegalMemberException {
 +    if (value == null) {
 +      return null;
 +    }
 +    try {
 +      switch (this.getDataType()) {
 +        case StringType:
 +          try {
 +            return new BigDecimal(value.toString());
 +          } catch (NumberFormatException e) {
-             throw new FilterUnsupportedException(e);
++            throw new FilterIllegalMemberException(e);
 +          }
 +        case ShortType:
 +          return new BigDecimal((short) value);
 +        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(
++          throw new FilterIllegalMemberException(
 +              "Cannot convert" + this.getDataType().name() + " to Long type value");
 +      }
 +    } catch (ClassCastException e) {
-       throw new FilterUnsupportedException(
++      throw new FilterIllegalMemberException(
 +          "Cannot convert" + this.getDataType().name() + " to Long type value");
 +    }
 +
 +  }
 +
-   public Long getTime() throws FilterUnsupportedException {
++  public Long getTime() throws FilterIllegalMemberException {
 +    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));
++          // Currently the query engine layer only supports yyyy-MM-dd HH:mm:ss date format
++          // no matter in which format the data is been stored, so while retrieving the direct
++          // surrogate value for filter member first it should be converted in date form as per
++          // above format and needs to retrieve time stamp.
++          SimpleDateFormat parser =
++              new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
 +          Date dateToStr;
 +          try {
 +            dateToStr = parser.parse(value.toString());
 +            return dateToStr.getTime() * 1000;
 +          } catch (ParseException e) {
-             throw new FilterUnsupportedException(
++            throw new FilterIllegalMemberException(
 +                "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
 +          }
 +        case ShortType:
 +          return ((Short) value).longValue();
 +        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(
++          throw new FilterIllegalMemberException(
 +              "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
 +      }
 +    } catch (ClassCastException e) {
-       throw new FilterUnsupportedException(
++      throw new FilterIllegalMemberException(
 +          "Cannot convert" + this.getDataType().name() + " to Time/Long type value");
 +    }
 +
 +  }
 +
-   public Boolean getBoolean() throws FilterUnsupportedException {
++  public Boolean getBoolean() throws FilterIllegalMemberException {
 +    if (value == null) {
 +      return null;
 +    }
 +    try {
 +      switch (this.getDataType()) {
 +        case StringType:
 +          try {
 +            return Boolean.parseBoolean(value.toString());
 +          } catch (NumberFormatException e) {
-             throw new FilterUnsupportedException(e);
++            throw new FilterIllegalMemberException(e);
 +          }
 +
 +        case BooleanType:
 +          return Boolean.parseBoolean(value.toString());
 +
 +        default:
-           throw new FilterUnsupportedException(
++          throw new FilterIllegalMemberException(
 +              "Cannot convert" + this.getDataType().name() + " to boolean type value");
 +      }
 +    } catch (ClassCastException e) {
-       throw new FilterUnsupportedException(
++      throw new FilterIllegalMemberException(
 +          "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() {
++  public List<String> getListAsString() throws FilterIllegalMemberException {
 +    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 ShortType:
 +          result = this.getShort().equals(objToCompare.getShort());
 +          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) {
++    } catch (FilterIllegalMemberException ex) {
 +      return false;
 +    }
 +
 +    return result;
 +  }
 +
 +  public boolean isNull() {
 +    return value == null;
 +  }
 +
 +  @Override public int compareTo(ExpressionResult o) {
 +    try {
 +      switch (o.dataType) {
 +        case ShortType:
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
index 62bab78,0000000..5bc1373
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
@@@ -1,84 -1,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.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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        addExprRightRes.set(DataType.ShortType, val1.getShort() + val2.getShort());
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
index d98b301,0000000..3e858e2
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
@@@ -1,84 -1,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.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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        divideExprRightRes.set(DataType.ShortType, val1.getShort() / val2.getShort());
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
index 7f8d8aa,0000000..72b45c7
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
@@@ -1,86 -1,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.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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        multiplyExprRightRes.set(DataType.ShortType, val1.getShort() * val2.getShort());
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
index 0e7ab2e,0000000..b65b032
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
@@@ -1,86 -1,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.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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        subtractExprRightRes.set(DataType.ShortType, val1.getShort() - val2.getShort());
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
index 80ad32e,0000000..ce8be43
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
@@@ -1,98 -1,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 org.carbondata.scan.expression.DataType;
 +import org.carbondata.scan.expression.Expression;
 +import org.carbondata.scan.expression.ExpressionResult;
++import org.carbondata.scan.expression.exception.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        result = val1.getShort().equals(val2.getShort());
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
index 4da578b,0000000..5318408
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
@@@ -1,88 -1,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.FilterIllegalMemberException;
 +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 {
++  public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        result = elRes.getShort() >= (erRes.getShort());
 +        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() + ')';
 +  }
 +}


[11/50] [abbrv] incubator-carbondata git commit: Removed unused properties from carbon (#739)

Posted by ch...@apache.org.
Removed unused properties from carbon (#739)

* Removed unused properties from carbon

* Removed unused carbon properties

* Fixed review comments


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

Branch: refs/heads/master
Commit: 60490179674681985aad2465afd8c0f5b025e185
Parents: e96de9f
Author: nareshpr <pr...@gmail.com>
Authored: Sat Jun 25 14:28:35 2016 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sat Jun 25 14:28:35 2016 +0530

----------------------------------------------------------------------
 conf/carbon.properties.template                 |  94 ++++++
 .../core/constants/CarbonCommonConstants.java   | 226 +------------
 .../core/datastorage/util/StoreFactory.java     |  36 +--
 .../carbondata/core/util/CarbonProperties.java  | 314 -------------------
 .../org/carbondata/core/util/CarbonUtil.java    |  35 +--
 dev/molap.properties.template                   |  94 ------
 dev/molap.properties_spark                      |  90 ------
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |  82 +----
 .../store/CarbonFactDataHandlerColumnar.java    |  10 +-
 9 files changed, 113 insertions(+), 868 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/conf/carbon.properties.template
----------------------------------------------------------------------
diff --git a/conf/carbon.properties.template b/conf/carbon.properties.template
new file mode 100644
index 0000000..314320c
--- /dev/null
+++ b/conf/carbon.properties.template
@@ -0,0 +1,94 @@
+#################### System Configuration ##################
+#Mandatory. Carbon Store path
+carbon.storelocation=hdfs://hacluster/Opt/CarbonStore
+#Base directory for Data files
+carbon.ddl.base.hdfs.url=hdfs://hacluster/opt/data
+#Path where the bad records are stored
+carbon.badRecords.location=/opt/Carbon/Spark/badrecords
+#Mandatory. path to kettle home
+carbon.kettle.home=$<SPARK_HOME>/carbonlib/carbonplugins
+
+#################### Performance Configuration ##################
+######## DataLoading Configuration ########
+#File read buffer size used during sorting:MIN=:MAX=
+carbon.sort.file.buffer.size=20
+#Rowset size exchanged between data load graph steps.:MIN=:MAX=
+carbon.graph.rowset.size=100000
+#Number of cores to be used while data loading:MIN=:MAX=
+carbon.number.of.cores.while.loading=6
+#CARBON sort size.:MIN=:MAX=
+carbon.sort.size=500000
+#Algorithm for hashmap for hashkey calculation
+carbon.enableXXHash=true
+#Number of cores to be used for block sort while dataloading
+#carbon.number.of.cores.block.sort=7
+#max level cache size upto which level cache will be loaded in memory
+#carbon.max.level.cache.size=-1
+#enable prefetch of data during merge sort while reading data from sort temp files in data loading
+#carbon.merge.sort.prefetch=true
+######## Compaction Configuration ########
+#Number of cores to be used while compacting:MIN=:MAX=
+carbon.number.of.cores.while.compacting=2
+#default minor compaction in MBs
+carbon.minor.compaction.size=256
+#default major compaction in MBs
+carbon.major.compaction.size=1024
+######## Query Configuration ########
+#Number of cores to be used.:MIN=:MAX=
+carbon.number.of.cores=4
+#Carbon Inmemory record size:MIN=:MAX=
+carbon.inmemory.record.size=100000
+#Improves the performance of filter query
+carbon.enable.quick.filter=false
+
+#################### Extra Configuration ##################
+##Timestamp format of input data used for timestamp data type.
+#carbon.timestamp.format=yyyy-MM-dd HH:mm:ss
+######## Dataload Configuration ########
+######File write buffer size used during sorting.
+#carbon.sort.file.write.buffer.size=10485760
+##Minimum no of intermediate files after which sort merged to be started.
+#carbon.sort.intermediate.files.limit=20
+##space reserved in percentage for writing block meta data in carbon data file
+#carbon.block.meta.size.reserved.percentage=10
+##csv reading buffer size.
+#carbon.csv.read.buffersize.byte=1048576
+##space reserved in percentage for writing block meta data in carbon data file
+#carbon.block.meta.size.reserved.percentage=10
+##High Cardinality value
+#high.cardinality.value=100000
+##CARBON maximum no of threads used for sorting.
+#carbon.max.thread.for.sorting=3
+##Carbon blocklet size. Note: this configuration cannot be change once store is generated
+#carbon.blocklet.size=120000
+##How to times retry to get the lock
+#carbon.load.metadata.lock.retries=3
+##Maximum number of blocklets written in a single file.:Min=1:Max=1000
+#carbon.max.file.size=100
+##Interval between the retries to get the lock
+#carbon.load.metadata.lock.retry.timeout.sec=5
+##Temporary store location, By default it will take System.getProperty("java.io.tmpdir")
+#carbon.tempstore.location=/opt/Carbon/TempStoreLoc
+##data loading records count logger
+#carbon.load.log.counter=500000
+######## Compaction Configuration ########
+##to specify number of segments to be preserved from compaction
+#carbon.numberof.preserve.segments=0
+##To determine the loads of number of days to be compacted
+#carbon.allowed.compaction.days=0
+##To enable compaction while data loading
+#carbon.enable.auto.load.merge=false
+######## Query Configuration ########
+##Maximum time allowed for one query to be executed.
+#max.query.execution.time=60
+##Min max is feature added to enhance query performance. To disable this feature, make it false.
+#carbon.enableMinMax=true
+##number of core to load the blocks in driver
+#no.of.cores.to.load.blocks.in.driver=10
+######## Global Dictionary Configurations ########
+##To enable/disable identify high cardinality during first data loading
+#high.cardinality.identify.enable=true
+##threshold to identify whether high cardinality column
+#high.cardinality.threshold=1000000
+##Percentage to identify whether column cardinality is more than configured percent of total row count
+#high.cardinality.row.count.percentage=80
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/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 cd25b88..a120d00 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -21,11 +21,6 @@ package org.carbondata.core.constants;
 
 public final class CarbonCommonConstants {
   /**
-   * MERGER_FOLDER_EXT
-   */
-  public static final String MERGER_FOLDER_EXT = ".merge";
-
-  /**
    * integer size in bytes
    */
   public static final int INT_SIZE_IN_BYTE = 4;
@@ -38,22 +33,6 @@ public final class CarbonCommonConstants {
    */
   public static final int DOUBLE_SIZE_IN_BYTE = 8;
   /**
-   * ONLINE_MERGE_MIN_VALUE
-   */
-  public static final int ONLINE_MERGE_MIN_VALUE = 10;
-  /**
-   * ONLINE_MERGE_MAX_VALUE
-   */
-  public static final int ONLINE_MERGE_MAX_VALUE = 100;
-  /**
-   * OFFLINE_MERGE_MIN_VALUE
-   */
-  public static final int OFFLINE_MERGE_MIN_VALUE = 100;
-  /**
-   * OFFLINE_MERGE_MAX_VALUE
-   */
-  public static final int OFFLINE_MERGE_MAX_VALUE = 500;
-  /**
    * LONG size in bytes
    */
   public static final int LONG_SIZE_IN_BYTE = 8;
@@ -74,30 +53,6 @@ public final class CarbonCommonConstants {
    */
   public static final String STORE_LOCATION = "carbon.storelocation";
   /**
-   * The keystore type
-   */
-  public static final String KEYSTORE_TYPE = "carbon.keystore.type";
-  /**
-   * The value store type
-   */
-  public static final String VALUESTORE_TYPE = "carbon.valuestore.type";
-  /**
-   * online merge file size
-   */
-  public static final String ONLINE_MERGE_FILE_SIZE = "carbon.online.merge.file.size";
-  /**
-   * online merge file size default value
-   */
-  public static final String ONLINE_MERGE_FILE_SIZE_DEFAULT_VALUE = "10";
-  /**
-   * offline merge file size
-   */
-  public static final String OFFLINE_MERGE_FILE_SIZE = "carbon.offline.merge.file.size";
-  /**
-   * offline merge file size default value
-   */
-  public static final String OFFLINE_MERGE_FILE_SIZE_DEFAULT_VALUE = "100";
-  /**
    * blocklet size in carbon file
    */
   public static final String BLOCKLET_SIZE = "carbon.blocklet.size";
@@ -110,41 +65,17 @@ public final class CarbonCommonConstants {
    */
   public static final String NUM_CORES = "carbon.number.of.cores";
   /**
-   * carbon batchsize
-   */
-  public static final String BATCH_SIZE = "carbon.batch.size";
-  /**
-   * CARDINALITY_INCREMENT_VALUE
-   */
-  public static final String CARDINALITY_INCREMENT_VALUE = "carbon.cardinality.increment.value";
-  /**
    * carbon sort size
    */
   public static final String SORT_SIZE = "carbon.sort.size";
   /**
    * default location of the carbon member, hierarchy and fact files
    */
-  public static final String STORE_LOCATION_DEFAULT_VAL = "../unibi-solutions/system/carbon/store";
-  /**
-   * default keystore type
-   */
-  public static final String KEYSTORE_TYPE_DEFAULT_VAL = "COMPRESSED_SINGLE_ARRAY";
-  /**
-   * default value store type
-   */
-  public static final String VALUESTORE_TYPE_DEFAULT_VAL = "HEAVY_VALUE_COMPRESSION";
-  /**
-   * CARDINALITY_INCREMENT_DEFAULT_VALUE
-   */
-  public static final String CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL = "10";
-  /**
-   * CARDINALITY_INCREMENT_MIN_VALUE
-   */
-  public static final int CARDINALITY_INCREMENT_MIN_VAL = 5;
+  public static final String STORE_LOCATION_DEFAULT_VAL = "../carbon.store";
   /**
    * CARDINALITY_INCREMENT_DEFAULT_VALUE
    */
-  public static final int CARDINALITY_INCREMENT_MAX_VAL = 30;
+  public static final int CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL = 10;
   /**
    * default blocklet size
    */
@@ -183,18 +114,6 @@ public final class CarbonCommonConstants {
    */
   public static final int NUM_CORES_MAX_VAL = 32;
   /**
-   * default carbon batchsize
-   */
-  public static final String BATCH_SIZE_DEFAULT_VAL = "1000";
-  /**
-   * min carbon batchsize
-   */
-  public static final int BATCH_SIZE_MIN_VAL = 500;
-  /**
-   * max carbon batchsize
-   */
-  public static final int BATCH_SIZE_MAX_VAL = 100000;
-  /**
    * default carbon sort size
    */
   public static final String SORT_SIZE_DEFAULT_VAL = "100000";
@@ -207,19 +126,10 @@ public final class CarbonCommonConstants {
    */
   public static final String CARBON_PROPERTIES_FILE_PATH = "../../../conf/carbon.properties";
   /**
-   * CARBON_BADRECORDS_ENCRYPTION
-   */
-  public static final String CARBON_BADRECORDS_ENCRYPTION = "carbon.badRecords.encryption";
-  /**
    * CARBON_DDL_BASE_HDFS_URL
    */
   public static final String CARBON_DDL_BASE_HDFS_URL = "carbon.ddl.base.hdfs.url";
   /**
-   * CARBON_BADRECORDS_ENCRYPTION_DEFAULT_VAL
-   */
-  public static final String CARBON_BADRECORDS_ENCRYPTION_DEFAULT_VAL = "false";
-
-  /**
    * Slice Meta data file.
    */
   public static final String SLICE_METADATA_FILENAME = "sliceMetaData";
@@ -298,57 +208,6 @@ public final class CarbonCommonConstants {
    */
   public static final String SORT_TEMP_FILE_LOCATION = "sortrowtmp";
   /**
-   * SORT_BUFFER_SIZE
-   */
-  public static final String SORT_BUFFER_SIZE = "carbon.sort.buffer.size";
-  /**
-   * SORT_BUFFER_SIZE_DEFAULT_SIZE
-   */
-  public static final String SORT_BUFFER_SIZE_DEFAULT_VALUE = "5000";
-  /**
-   * SORT_BUFFER_SIZE_MIN_SIZE
-   */
-  public static final int SORT_BUFFER_SIZE_MIN_VALUE = 5;
-  /**
-   * DATA_LOAD_Q_SIZE
-   */
-  public static final String DATA_LOAD_Q_SIZE = "carbon.dataload.queuesize";
-  /**
-   * DATA_LOAD_Q_SIZE_DEFAULT
-   */
-  public static final String DATA_LOAD_Q_SIZE_DEFAULT = "100";
-  /**
-   * DATA_LOAD_Q_SIZE_MIN
-   */
-  public static final int DATA_LOAD_Q_SIZE_MIN = 1;
-  /**
-   * DATA_LOAD_Q_SIZE_MAX
-   */
-  public static final int DATA_LOAD_Q_SIZE_MAX = 100;
-
-  /**
-   * DATA_LOAD_CONC_EXE_SIZE
-   */
-  public static final String DATA_LOAD_CONC_EXE_SIZE = "carbon.dataload.concurrent.execution.size";
-  /**
-   * DATA_LOAD_CONC_EXE_SIZE_DEFAULT
-   */
-  public static final String DATA_LOAD_CONC_EXE_SIZE_DEFAULT = "1";
-  /**
-   * DATA_LOAD_CONC_EXE_SIZE_MIN
-   */
-  public static final int DATA_LOAD_CONC_EXE_SIZE_MIN = 1;
-  /**
-   * DATA_LOAD_CONC_EXE_SIZE_MAX
-   */
-  public static final int DATA_LOAD_CONC_EXE_SIZE_MAX = 5;
-  /**
-   * CARBON_Realtime_data
-   */
-  public static final String CARBON_REALTIMEDATA_FILE =
-      "../unibi-solutions/system/carbon/realtimedata.properties";
-
-  /**
    * CARBON_RESULT_SIZE_DEFAULT
    */
   public static final String LEVEL_FILE_EXTENSION = ".level";
@@ -418,10 +277,6 @@ public final class CarbonCommonConstants {
    */
   public static final String CARBON_SORT_FILE_WRITE_BUFFER_SIZE_DEFAULT_VALUE = "50000";
   /**
-   * WRITE_ALL_NODE_IN_SINGLE_TIME_DEFAULT_VALUE
-   */
-  public static final String WRITE_ALL_NODE_IN_SINGLE_TIME_DEFAULT_VALUE = "true";
-  /**
    * Number of cores to be used while loading
    */
   public static final String NUM_CORES_LOADING = "carbon.number.of.cores.while.loading";
@@ -486,35 +341,14 @@ public final class CarbonCommonConstants {
    */
   public static final String HYPHEN_SPC_CHARACTER = "-#!:HYPHEN:!#-";
   /**
-   * CARBON_DECIMAL_POINTERS
-   */
-  public static final String CARBON_DECIMAL_POINTERS = "carbon.decimal.pointers";
-  /**
    * CARBON_DECIMAL_POINTERS_DEFAULT
    */
-  public static final String CARBON_DECIMAL_POINTERS_DEFAULT = "5";
-  /**
-   * CARBON_DECIMAL_POINTERS_AGG
-   */
-  public static final String CARBON_DECIMAL_POINTERS_AGG = "carbon.decimal.pointers.agg";
-  /**
-   * CARBON_DECIMAL_POINTERS_AGG_DEFAULT
-   */
-  public static final String CARBON_DECIMAL_POINTERS_AGG_DEFAULT = "4";
+  public static final byte CARBON_DECIMAL_POINTERS_DEFAULT = 5;
   /**
    * SORT_TEMP_FILE_EXT
    */
   public static final String SORT_TEMP_FILE_EXT = ".sorttemp";
   /**
-   * CARBON_SEQ_GEN_INMEMORY_LRU_CACHE_ENABLED
-   */
-  public static final String CARBON_SEQ_GEN_INMEMORY_LRU_CACHE_ENABLED =
-      "carbon.seqgen.inmemory.lru.cache.enabled";
-  /**
-   * CARBON_SEQ_GEN_INMEMORY_LRU_CACHE_ENABLED_DEFAULT_VALUE
-   */
-  public static final String CARBON_SEQ_GEN_INMEMORY_LRU_CACHE_ENABLED_DEFAULT_VALUE = "false";
-  /**
    * CARBON_MAX_THREAD_FOR_SORTING
    */
   public static final String CARBON_MAX_THREAD_FOR_SORTING = "carbon.max.thread.for.sorting";
@@ -545,15 +379,6 @@ public final class CarbonCommonConstants {
    */
   public static final int DEFAULT_COLLECTION_SIZE = 16;
   /**
-   * CARBON_DATALOAD_VALID_CSVFILE_SIZE
-   */
-  public static final String CARBON_DATALOAD_VALID_CSVFILE_SIZE =
-      "carbon.dataload.valid.csvfile.size(in GB)";
-  /**
-   * CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE
-   */
-  public static final String CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE = "5";
-  /**
    * CARBON_TIMESTAMP_DEFAULT_FORMAT
    */
   public static final String CARBON_TIMESTAMP_DEFAULT_FORMAT = "yyyy-MM-dd HH:mm:ss";
@@ -562,15 +387,6 @@ public final class CarbonCommonConstants {
    */
   public static final String CARBON_TIMESTAMP_FORMAT = "carbon.timestamp.format";
   /**
-   * CARBON_DATALOAD_VALID_CSVFILE_SIZE
-   */
-  public static final String CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE =
-      "carbon.dataload.csv.filecount";
-  /**
-   * CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE
-   */
-  public static final String CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE_DEFAULTVALUE = "100";
-  /**
    * STORE_LOCATION_HDFS
    */
   public static final String STORE_LOCATION_HDFS = "carbon.storelocation.hdfs";
@@ -751,14 +567,6 @@ public final class CarbonCommonConstants {
    */
   public static final String CARBON_MAX_LEVEL_CACHE_SIZE_DEFAULT = "-1";
   /**
-   * retry interval after which loading a level file will be retried
-   */
-  public static final String CARBON_LOAD_LEVEL_RETRY_INTERVAL = "Carbon.load.level.retry.interval";
-  /**
-   * retry interval default value
-   */
-  public static final String CARBON_LOAD_LEVEL_RETRY_INTERVAL_DEFAULT = "12";
-  /**
    * DOUBLE_VALUE_MEASURE
    */
   public static final char SUM_COUNT_VALUE_MEASURE = 'n';
@@ -781,8 +589,6 @@ public final class CarbonCommonConstants {
    * for dimensions , one of ignore dictionary dimensions , one for measures.
    */
   public static final int ARRAYSIZE = 3;
-  public static final String CARBON_UNIFIED_STORE_PATH = "carbon.unified.store.path";
-  public static final String CARBON_UNIFIED_STORE_PATH_DEFAULT = "false";
   /**
    * CARBON_PREFETCH_BUFFERSIZE
    */
@@ -796,22 +602,6 @@ public final class CarbonCommonConstants {
    */
   public static final String TEMPWRITEFILEEXTENSION = ".write";
   /**
-   * MERGE_THRESHOLD_VALUE
-   */
-  public static final String MERGE_THRESHOLD_VALUE = "carbon.merge.threshold";
-  /**
-   * MERGE_THRESHOLD_DEFAULT_VAL
-   */
-  public static final String MERGE_THRESHOLD_DEFAULT_VAL = "10";
-  /**
-   * TO_LOAD_MERGE_MAX_SIZE
-   */
-  public static final String TO_LOAD_MERGE_MAX_SIZE = "to.merge.load.max.size";
-  /**
-   * TO_LOAD_MERGE_MAX_SIZE_DEFAULT
-   */
-  public static final String TO_LOAD_MERGE_MAX_SIZE_DEFAULT = "1";
-  /**
    * ENABLE_AUTO_LOAD_MERGE
    */
   public static final String ENABLE_AUTO_LOAD_MERGE = "carbon.enable.auto.load.merge";
@@ -868,16 +658,6 @@ public final class CarbonCommonConstants {
    */
   public static final int INVALID_SURROGATE_KEY = -1;
 
-  /**
-   * table split partition
-   */
-  public static final String TABLE_SPLIT_PARTITION = "carbon.table.split.partition.enable";
-
-  /**
-   * table split partition default value
-   */
-  public static final String TABLE_SPLIT_PARTITION_DEFAULT_VALUE = "false";
-
   public static final String INVALID_SEGMENT_ID = "-1";
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/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..63f8157 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
@@ -19,7 +19,6 @@
 
 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;
@@ -38,22 +37,9 @@ import org.carbondata.core.datastorage.store.impl.key.compressed.CompressedSingl
 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;
@@ -63,26 +49,8 @@ public final class StoreFactory {
   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;
-    } else {
-      valueType = StoreType.HEAVY_VALUE_COMPRESSION;
-    }
+    keyType = StoreType.COMPRESSED_SINGLE_ARRAY;
+    valueType = StoreType.HEAVY_VALUE_COMPRESSION;
   }
 
   private StoreFactory() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
index a337ac4..e6a1ce9 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
@@ -75,34 +75,12 @@ public final class CarbonProperties {
           CarbonCommonConstants.STORE_LOCATION_DEFAULT_VAL);
     }
 
-    if (null == carbonProperties.getProperty(CarbonCommonConstants.VALUESTORE_TYPE)) {
-      carbonProperties.setProperty(CarbonCommonConstants.VALUESTORE_TYPE,
-          CarbonCommonConstants.VALUESTORE_TYPE_DEFAULT_VAL);
-    }
-
-    if (null == carbonProperties.getProperty(CarbonCommonConstants.KEYSTORE_TYPE)) {
-      carbonProperties.setProperty(CarbonCommonConstants.KEYSTORE_TYPE,
-          CarbonCommonConstants.KEYSTORE_TYPE_DEFAULT_VAL);
-    }
-
     validateBlockletSize();
     validateMaxFileSize();
     validateNumCores();
     validateNumCoresBlockSort();
-    validateBatchSize();
     validateSortSize();
-    validateCardinalityIncrementValue();
-    validateOnlineMergerSize();
-    validateOfflineMergerSize();
-    validateSortBufferSize();
-    validateDataLoadQSize();
-    validateDataLoadConcExecSize();
-    validateDecimalPointers();
-    validateDecimalPointersAgg();
-    validateCsvFileSize();
-    validateNumberOfCsvFile();
     validateBadRecordsLocation();
-    validateBadRecordsEncryption();
     validateHighCardinalityIdentify();
     validateHighCardinalityThreshold();
     validateHighCardinalityInRowCountPercentage();
@@ -117,168 +95,6 @@ public final class CarbonProperties {
     }
   }
 
-  private void validateBadRecordsEncryption() {
-    String badRecordsEncryption =
-        carbonProperties.getProperty(CarbonCommonConstants.CARBON_BADRECORDS_ENCRYPTION);
-    if (null == badRecordsEncryption || badRecordsEncryption.length() == 0) {
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_BADRECORDS_ENCRYPTION,
-          CarbonCommonConstants.CARBON_BADRECORDS_ENCRYPTION_DEFAULT_VAL);
-    }
-  }
-
-  private void validateCsvFileSize() {
-    try {
-      int csvFileSizeProperty = Integer.parseInt(carbonProperties
-          .getProperty(CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE,
-              CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE));
-      if (csvFileSizeProperty < 1) {
-        LOGGER.info("Invalid value for " + CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE
-                + "\"Only Positive Integer(greater than zero) is allowed. Using the default value\""
-                + CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE);
-
-        carbonProperties.setProperty(CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE,
-            CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("Invalid value for " + CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE
-              + "\"Only Positive Integer(greater than zero) is allowed. Using the default value\""
-              + CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE);
-
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE,
-          CarbonCommonConstants.CARBON_DATALOAD_VALID_CSVFILE_SIZE_DEFAULTVALUE);
-    }
-  }
-
-  private void validateNumberOfCsvFile() {
-    String count = CarbonCommonConstants.CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE;
-    try {
-      int csvFileSizeProperty = Integer.parseInt(carbonProperties
-          .getProperty(count,
-              CarbonCommonConstants.CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE_DEFAULTVALUE));
-      if (csvFileSizeProperty < 1) {
-        LOGGER.info("Invalid value for " + count
-                + "\"Only Positive Integer(greater than zero) is allowed. Using the default value\""
-                + CarbonCommonConstants.CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE_DEFAULTVALUE);
-
-        carbonProperties.setProperty(count,
-            CarbonCommonConstants.CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE_DEFAULTVALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("Invalid value for " + count
-              + "\"Only Positive Integer(greater than zero) is allowed. Using the default value\""
-              + CarbonCommonConstants.CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE_DEFAULTVALUE);
-
-      carbonProperties.setProperty(count,
-          CarbonCommonConstants.CARBON_DATALOAD_VALID_NUMBAER_OF_CSVFILE_DEFAULTVALUE);
-    }
-  }
-
-  /**
-   * This method validates the batch size
-   */
-  private void validateOnlineMergerSize() {
-    String onlineMergeSize = carbonProperties
-        .getProperty(CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE,
-            CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-    try {
-      int offlineMergerSize = Integer.parseInt(onlineMergeSize);
-
-      if (offlineMergerSize < CarbonCommonConstants.ONLINE_MERGE_MIN_VALUE
-          || offlineMergerSize > CarbonCommonConstants.ONLINE_MERGE_MAX_VALUE) {
-        LOGGER.info("The online Merge Size value \"" + onlineMergeSize
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-        carbonProperties.setProperty(CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE,
-            CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The online Merge Size value \"" + onlineMergeSize
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-      carbonProperties.setProperty(CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE,
-          CarbonCommonConstants.ONLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-    }
-  }
-
-  /**
-   * This method validates the batch size
-   */
-  private void validateOfflineMergerSize() {
-    String offLineMergerSize = carbonProperties
-        .getProperty(CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE,
-            CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-    try {
-      int offLineMergeSize = Integer.parseInt(offLineMergerSize);
-
-      if (offLineMergeSize < CarbonCommonConstants.OFFLINE_MERGE_MIN_VALUE
-          || offLineMergeSize > CarbonCommonConstants.OFFLINE_MERGE_MAX_VALUE) {
-        LOGGER.info("The offline Merge Size value \"" + offLineMergerSize
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-        carbonProperties.setProperty(CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE,
-            CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The offline Merge Size value \"" + offLineMergerSize
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-      carbonProperties.setProperty(CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE,
-          CarbonCommonConstants.OFFLINE_MERGE_FILE_SIZE_DEFAULT_VALUE);
-    }
-  }
-
-  /**
-   * This method validates the batch size
-   */
-  private void validateBatchSize() {
-    String batchSizeStr = carbonProperties.getProperty(CarbonCommonConstants.BATCH_SIZE,
-        CarbonCommonConstants.BATCH_SIZE_DEFAULT_VAL);
-    try {
-      int batchSize = Integer.parseInt(batchSizeStr);
-
-      if (batchSize < CarbonCommonConstants.BATCH_SIZE_MIN_VAL
-          || batchSize > CarbonCommonConstants.BATCH_SIZE_MAX_VAL) {
-        LOGGER.info("The batch size value \"" + batchSizeStr + "\" is invalid. "
-            + "Using the default value \"" + CarbonCommonConstants.BATCH_SIZE_DEFAULT_VAL);
-        carbonProperties.setProperty(CarbonCommonConstants.BATCH_SIZE,
-            CarbonCommonConstants.BATCH_SIZE_DEFAULT_VAL);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The batch size value \"" + batchSizeStr
-          + "\" is invalid. Using the default value \""
-          + CarbonCommonConstants.BATCH_SIZE_DEFAULT_VAL);
-      carbonProperties.setProperty(CarbonCommonConstants.BATCH_SIZE,
-          CarbonCommonConstants.BATCH_SIZE_DEFAULT_VAL);
-    }
-  }
-
-  /**
-   * This method validates the batch size
-   */
-  private void validateCardinalityIncrementValue() {
-    String cardinalityIncr = carbonProperties
-        .getProperty(CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE,
-            CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL);
-    try {
-      int batchSize = Integer.parseInt(cardinalityIncr);
-
-      if (batchSize < CarbonCommonConstants.CARDINALITY_INCREMENT_MIN_VAL
-          || batchSize > CarbonCommonConstants.CARDINALITY_INCREMENT_MAX_VAL) {
-        LOGGER.info("The batch size value \"" + cardinalityIncr
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL);
-        carbonProperties.setProperty(CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE,
-            CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The cardinality size value \"" + cardinalityIncr
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.BATCH_SIZE_DEFAULT_VAL);
-      carbonProperties.setProperty(CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE,
-          CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL);
-    }
-  }
-
   /**
    * This method validates the blocklet size
    */
@@ -306,111 +122,6 @@ public final class CarbonProperties {
   }
 
   /**
-   * This method validates data load queue size
-   */
-  private void validateDataLoadQSize() {
-    String dataLoadQSize = carbonProperties.getProperty(CarbonCommonConstants.DATA_LOAD_Q_SIZE,
-        CarbonCommonConstants.DATA_LOAD_Q_SIZE_DEFAULT);
-    try {
-      int dataLoadQSizeInt = Integer.parseInt(dataLoadQSize);
-
-      if (dataLoadQSizeInt < CarbonCommonConstants.DATA_LOAD_Q_SIZE_MIN
-          || dataLoadQSizeInt > CarbonCommonConstants.DATA_LOAD_Q_SIZE_MAX) {
-        LOGGER.info("The data load queue size value \"" + dataLoadQSize
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.DATA_LOAD_Q_SIZE_DEFAULT);
-        carbonProperties.setProperty(CarbonCommonConstants.DATA_LOAD_Q_SIZE,
-            CarbonCommonConstants.DATA_LOAD_Q_SIZE_DEFAULT);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The data load queue size value \"" + dataLoadQSize
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.DATA_LOAD_Q_SIZE_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.DATA_LOAD_Q_SIZE,
-          CarbonCommonConstants.DATA_LOAD_Q_SIZE_DEFAULT);
-    }
-  }
-
-  /**
-   * This method validates the data load concurrent exec size
-   */
-  private void validateDataLoadConcExecSize() {
-    String dataLoadConcExecSize = carbonProperties
-        .getProperty(CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE,
-            CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_DEFAULT);
-    try {
-      int dataLoadConcExecSizeInt = Integer.parseInt(dataLoadConcExecSize);
-
-      if (dataLoadConcExecSizeInt < CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_MIN
-          || dataLoadConcExecSizeInt > CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_MAX) {
-        LOGGER.info("The data load concurrent exec size value \"" + dataLoadConcExecSize
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_DEFAULT);
-        carbonProperties.setProperty(CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE,
-            CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_DEFAULT);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The data load concurrent exec size value \"" + dataLoadConcExecSize
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE,
-          CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_DEFAULT);
-    }
-  }
-
-  /**
-   * This method validates the decimal pointers size
-   */
-  private void validateDecimalPointers() {
-    String decimalPointers = carbonProperties
-        .getProperty(CarbonCommonConstants.CARBON_DECIMAL_POINTERS,
-            CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT);
-    try {
-      int decimalPointersInt = Integer.parseInt(decimalPointers);
-
-      if (decimalPointersInt < 0 || decimalPointersInt > 15) {
-        LOGGER.info("The decimal pointers agg \"" + decimalPointers
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_DEFAULT);
-        carbonProperties.setProperty(CarbonCommonConstants.CARBON_DECIMAL_POINTERS,
-            CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The decimal pointers agg \"" + decimalPointers
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.DATA_LOAD_CONC_EXE_SIZE_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_DECIMAL_POINTERS,
-          CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT);
-    }
-  }
-
-  /**
-   * This method validates the data load concurrent exec size
-   */
-  private void validateDecimalPointersAgg() {
-    String decimalPointers = carbonProperties
-        .getProperty(CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG,
-            CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG_DEFAULT);
-    try {
-      int decimalPointersInt = Integer.parseInt(decimalPointers);
-
-      if (decimalPointersInt < 0 || decimalPointersInt > 15) {
-        LOGGER.info("The decimal pointers agg \"" + decimalPointers
-                + "\" is invalid. Using the default value \""
-                + CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG_DEFAULT);
-        carbonProperties.setProperty(CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG,
-            CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG_DEFAULT);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The decimal pointers agg \"" + decimalPointers
-              + "\" is invalid. Using the default value \""
-              + CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG_DEFAULT);
-      carbonProperties.setProperty(CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG,
-          CarbonCommonConstants.CARBON_DECIMAL_POINTERS_AGG_DEFAULT);
-    }
-  }
-
-  /**
    * TODO: This method validates the maximum number of blocklets per file ?
    */
   private void validateMaxFileSize() {
@@ -515,31 +226,6 @@ public final class CarbonProperties {
     }
   }
 
-  /**
-   * This method validates the sort size
-   */
-  private void validateSortBufferSize() {
-    String sortSizeStr = carbonProperties.getProperty(CarbonCommonConstants.SORT_BUFFER_SIZE,
-        CarbonCommonConstants.SORT_BUFFER_SIZE_DEFAULT_VALUE);
-    try {
-      int sortSize = Integer.parseInt(sortSizeStr);
-
-      if (sortSize < CarbonCommonConstants.SORT_BUFFER_SIZE_MIN_VALUE) {
-        LOGGER.info("The batch size value \"" + sortSizeStr
-            + "\" is invalid. Using the default value \""
-            + CarbonCommonConstants.SORT_BUFFER_SIZE_DEFAULT_VALUE);
-        carbonProperties.setProperty(CarbonCommonConstants.SORT_BUFFER_SIZE,
-            CarbonCommonConstants.SORT_BUFFER_SIZE_DEFAULT_VALUE);
-      }
-    } catch (NumberFormatException e) {
-      LOGGER.info("The batch size value \"" + sortSizeStr
-          + "\" is invalid. Using the default value \""
-          + CarbonCommonConstants.SORT_BUFFER_SIZE_DEFAULT_VALUE);
-      carbonProperties.setProperty(CarbonCommonConstants.SORT_BUFFER_SIZE,
-          CarbonCommonConstants.SORT_BUFFER_SIZE_DEFAULT_VALUE);
-    }
-  }
-
   private void validateHighCardinalityIdentify() {
     String highcardIdentifyStr = carbonProperties.getProperty(
         CarbonCommonConstants.HIGH_CARDINALITY_IDENTIFY_ENABLE,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/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 2d50f07..0f94c5a 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -299,9 +299,7 @@ public final class CarbonUtil {
    */
   public static int[] getIncrementedCardinality(int[] dimCardinality) {
     // get the cardinality incr factor
-    final int incrValue = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE,
-            CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL));
+    final int incrValue = CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL;
 
     int perIncr = 0;
     int remainder = 0;
@@ -333,9 +331,7 @@ public final class CarbonUtil {
 
   public static int getIncrementedCardinality(int dimCardinality) {
     // get the cardinality incr factor
-    final int incrValue = Integer.parseInt(CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE,
-            CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL));
+    final int incrValue = CarbonCommonConstants.CARDINALITY_INCREMENT_VALUE_DEFAULT_VAL;
 
     int perIncr = 0;
     int remainder = 0;
@@ -1254,26 +1250,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
@@ -1296,13 +1272,6 @@ public final class CarbonUtil {
     }
     String basePath = prop.getProperty(CarbonCommonConstants.STORE_LOCATION,
         CarbonCommonConstants.STORE_LOCATION_DEFAULT_VAL);
-    String useUniquePath = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.CARBON_UNIFIED_STORE_PATH,
-            CarbonCommonConstants.CARBON_UNIFIED_STORE_PATH_DEFAULT);
-    if (null != schemaName && !schemaName.isEmpty() && null != cubeName && !cubeName.isEmpty()
-        && "true".equals(useUniquePath)) {
-      basePath = basePath + File.separator + schemaName + File.separator + cubeName;
-    }
     return basePath;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/dev/molap.properties.template
----------------------------------------------------------------------
diff --git a/dev/molap.properties.template b/dev/molap.properties.template
deleted file mode 100644
index 7aa69d4..0000000
--- a/dev/molap.properties.template
+++ /dev/null
@@ -1,94 +0,0 @@
-#################### Performance Configuration ##################
-#File read buffer size used during sorting:MIN=:MAX=
-carbon.sort.file.buffer.size=20
-#Rowset size exchanged between data load graph steps.:MIN=:MAX=
-carbon.graph.rowset.size=100000
-#Number of cores to be used.:MIN=:MAX=
-carbon.number.of.cores=4
-#Number of cores to be used while data loading:MIN=:MAX=
-carbon.number.of.cores.while.loading=6
-#Carbon Inmemory record size:MIN=:MAX=
-carbon.inmemory.record.size=100000
-#CARBON sort size.:MIN=:MAX=
-carbon.sort.size=500000
-#Improves the performance of filter query
-carbon.enable.quick.filter=false
-#Algorithm for hashmap for hashkey calculation
-carbon.enableXXHash=true
-
-#################### System Configuration ##################
-#Mandatory. Carbon Store path
-carbon.storelocation=hdfs://hacluster/Opt/CarbonStore
-#Base directory for Data files
-carbon.ddl.base.hdfs.url=hdfs://hacluster/opt/data
-#Path where the bad records are stored
-carbon.badRecords.location=/opt/Carbon/Spark/badrecords
-#To unify the carbon Cube and store path.
-carbon.unified.store.path=true
-#Mandatory. path to kettle home
-carbon.kettle.home=<SPARK_HOME>/lib/carbonplugins
-
-#################### Extra Configuration ##################
-######Carbon level write buffers in KB:MIN=:MAX=
-#carbon.level.write.bufferinkb=12238
-######File write buffer size used during sorting.
-#carbon.sort.file.write.buffer.size=10485760
-######Minimum no of intermediate files after which sort merged to be started.
-#carbon.sort.intermediate.files.limit=20
-######Number of threads for intermediate merging.
-#carbon.sort.intermedaite.number.of.threads=5
-######csv reading buffer size.
-#carbon.csv.read.buffersize.byte=1048576
-######High Cardinality value
-#high.cardinality.value =50000
-######Carbon blocklet size. Note: this configuration cannot be change once store is generated
-#carbon.blocklet.size=120000
-######CARBON maximum no of threads used for sorting.
-#carbon.max.thread.for.sorting=3
-######Maximum time allowed for one query to be executed.
-#max.query.execution.time=60
-######Aggregate table suggestion takes by number load for data sampling.
-#carbon.agg.loadCount=2
-######Number of partition to read
-#carbon.agg.partitionCount=1
-######Aggregate table suggestion takes number of fact file per load for data sampling.
-#carbon.agg.factCount=2
-######Aggregate table suggestion takes number of records per fact for data sampling.
-#carbon.agg.recordCount=5
-######benefitRatio =total records/no of records in aggregate table.if benefit ratio for aggregate combination is greater than configured value than it is selected for suggestion
-#carbon.agg.benefitRatio=10
-######Whether to cahe Datastats suggestion
-#carbon.agg.datastats.cache=false
-######Any query which takes more than configured value in seconds are considered for Aggregate suggestion.
-#carbon.agg.query.performance.goal=3
-######If this parameter is set to true, Carbon will cache the metadata on Server start up and reduce the first query execution time.NOTE: Curent descriptions not applicable when level cache is enabled.
-#carbon.agg.querystats.expiryday=30
-######If this parameter is set to true, Carbon will cache the metadata on Server start up and reduce the first query execution time.NOTE: Curent descriptions not applicable when level cache is enabled.
-#carbon.is.loadcube.startup=false
-######If this parameter is set to true, Carbon will cache the metadata after the successful data loading and reduce the first query execution time.NOTE: Curent descriptions not applicable when level cache is enabled.
-#carbon.is.loadcube.dataload=false
-######How to times retry to get the lock
-#carbon.load.metadata.lock.retries=3
-######Interval between the retries to get the lock
-#carbon.load.metadata.lock.retry.timeout.sec=5
-######Maximum number of blocklets written in a single file.:Min=1:Max=1000
-#carbon.max.file.size=100
-######Sort buffer size:MI=5:MAX=
-#carbon.sort.buffer.size=5000
-######Timestamp format of input data used for timestamp data type.
-#carbon.timestamp.format=yyyy-MM-dd HH:mm:ss
-######Cube is completely kept in memory.
-#carbon.forced.in.memory.cube=false
-######Maintains the complete cube cache in memory while data loading. Useful for increasing data load performance in case of history data loading. Set it to true for data load performance tuning.
-#carbon.seqgen.inmemory.lru.cache.enabled=false
-######Min max is feature added to enhance query performance. To disable this feature, make it false.
-#carbon.enableMinMax=true
-######Temporary store location, By default it will take System.getProperty("java.io.tmpdir")
-#carbon.tempstore.location=/opt/Carbon/TempStoreLoc
-
-
-#################### AUDIT LOGGING(Used when it is used without FI) ##################
-#carbon.auditlog.file.path=logs/CarbonAudit.log
-#carbon.auditlog.max.file.size=10MB
-#carbon.auditlog.max.backup.files=10
-#carbon.logging.level=INFO

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/dev/molap.properties_spark
----------------------------------------------------------------------
diff --git a/dev/molap.properties_spark b/dev/molap.properties_spark
deleted file mode 100644
index 0c564ff..0000000
--- a/dev/molap.properties_spark
+++ /dev/null
@@ -1,90 +0,0 @@
-carbon.schema.maxFileSize=50
-dataload.taskstatus.retention=2
-carbon.numberOfCubesToLoadConcurrent=5
-max.memory.threshold=60
-min.memory.threshold=50
-#1 means every day
-carbon.retention.schedule=1
-carbon.dataload.queuesize=100
-carbon.dataload.concurrent.execution.size=1
-carbon.result.limit=100000000
-mysql.null.value=\\N
-mssql.null.value=
-oracle.null.value=NULL
-carbon.sort.size=1000000
-carbon.queryexecutor.concurrent.execution.size=3
-#################### EXECUTION THREADS ##################
-carbon.number.of.cores=4
-carbon.smartJump.avoid.percent=70
-carbon.agg.enableXXHash=true
-carbon.spark.resultlimit=20000
-carbon.cache.used=false
-
-mysql.resultset.cursor.moveonly.forward=false
-
-carbon.level.write.bufferinkb=12238
-carbon.graph.rowset.size=100000
-carbon.sort.file.write.buffer.size=10485760
-carbon.sort.intermediate.files.limit=50
-carbon.sort.file.buffer.size=20
-carbon.sort.intermedaite.number.of.therads=5
-carbon.csv.read.buffersize.byte=1048576
-carbon.csv.read.copies=6
-carbon.datawriter.write.all.node=true
-carbon.data.load.log.counter=500000
-carbon.number.of.cores.while.loading=6
-carbon.prefetch.in.merge=true
-carbon.prefetch.bufferSize=20000
-carbon.inmemory.cache.use=true
-carbon.dataload.log.enabled=true
-
-
-## Spark CARBON related Properties
-#spark.dataset.location=../datasets_test/
-#spark.dp.location=../datapipelines_test/
-#spark.sqlconnections.location=../unibi-solutions/system/dbconnection/sqlconnections_test.xml
-#spark.url=local
-
-#carbon.storelocation=hdfs://master:54310/opt/ravi/store
-#carbon.storelocation=/opt/ravi/store1day
-carbon.storelocation=hdfs://master:54310/opt/ravi/perfstore
-#carbon.storelocation=/opt/ravi/store1day
-#carbon.storelocation=/opt/ravi/storebasecarbon
-#carbon.storelocation=/opt/ravi/storesinglenode
-
-
-
-
-spark.dataset.location=hdfs://master:54310/opt/ravi/sparkcarbon/datasets/
-spark.dp.location=hdfs://master:54310/opt/ravi/sparkcarbon/datapipelines/
-spark.sqlconnections.location=hdfs://master:54310/opt/ravi/sparkcarbon/sqlconnections/sqlconnections_test.xml
-spark.url=spark://master:7077
-spark.home=/opt/spark-1.0.0-rc3
-#spark.schema.path=/opt/ravi/steelwheels.carbon.xml
-spark.schema.path=/opt/ravi/PCC_Java.xml
-spark.schema.name=PCC
-spark.cube.name=ODM
-
-spark.executor.memory=200g
-spark.cores.max=76
-spark.usekryo.serializer=true
-spark.eventLog.enabled=true
-spark.sql.shuffle.partitions=200
-
-##### New properties for columnar ####################################################################
-# Enbale Columnar
-carbon.is.columnar.storage=true
-#Int or Short based indexes. use Int now (TODO  Short is not working) 
-is.int.based.indexer=true			
-#Store Unique Values for a column if not high cardinality dimension 
-aggregate.columnar.keyblock=true
-#Threshold for a dimension be considered High Cardinality 
-high.cardinality.value=100000
-#Numbers of tuples in Leaf  ( this can be 15x for columar store comared to row based store since each column is sperately read/decompressed) 
-carbon.leaf.node.size=120000
-#Use multiple of 8 bits for a colmn value
-carbon.is.fullyfilled.bits=true
-#To use NumberCompressor.java for compression . Since no benefit was found, keep it false
-is.compressed.keyblock=false
-#How many levels will be combined into one column .TODO only one supported
-carbon.dimension.split.value.in.columnar=1

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/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 b9a6649..56c6574 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
@@ -112,15 +112,6 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
 
   val metadata = loadMetadata(storePath)
 
-  lazy val useUniquePath = if ("true".equalsIgnoreCase(CarbonProperties.getInstance().
-    getProperty(
-      CarbonCommonConstants.CARBON_UNIFIED_STORE_PATH,
-      CarbonCommonConstants.CARBON_UNIFIED_STORE_PATH_DEFAULT))) {
-    true
-  } else {
-    false
-  }
-
   def lookupRelation1(
       databaseName: Option[String],
       tableName: String,
@@ -230,34 +221,8 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     }
     val fileType = FileFactory.getFileType(metadataPath)
     val metaDataBuffer = new ArrayBuffer[TableMeta]
-    if (useUniquePath) {
-      if (FileFactory.isFileExist(metadataPath, fileType)) {
-        val file = FileFactory.getCarbonFile(metadataPath, fileType)
-        val schemaFolders = file.listFiles()
-
-        schemaFolders.foreach(schemaFolder => {
-          if (schemaFolder.isDirectory) {
-            val cubeFolders = schemaFolder.listFiles()
-
-            cubeFolders.foreach(cubeFolder => {
-              val schemaPath = metadataPath + "/" + schemaFolder.getName + "/" + cubeFolder.getName
-              try {
-                fillMetaData(schemaPath, fileType, metaDataBuffer)
-                updateSchemasUpdatedTime(schemaFolder.getName, cubeFolder.getName)
-              } catch {
-                case ex: org.apache.hadoop.security.AccessControlException =>
-                // Ingnore Access control exception and get only accessible cube details
-              }
-            })
-          }
-        })
-      }
-
-    } else {
-
-      fillMetaData(metadataPath, fileType, metaDataBuffer)
-      updateSchemasUpdatedTime("", "")
-    }
+    fillMetaData(metadataPath, fileType, metaDataBuffer)
+    updateSchemasUpdatedTime("", "")
     MetaData(metaDataBuffer)
 
   }
@@ -605,13 +570,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
 
   private def getTimestampFileAndType(schemaName: String, cubeName: String) = {
 
-    val timestampFile = if (useUniquePath) {
-      storePath + "/" + schemaName + "/" + cubeName + "/" +
-      CarbonCommonConstants.SCHEMAS_MODIFIED_TIME_FILE
-    }
-    else {
-      storePath + "/" + CarbonCommonConstants.SCHEMAS_MODIFIED_TIME_FILE
-    }
+    val timestampFile = storePath + "/" + CarbonCommonConstants.SCHEMAS_MODIFIED_TIME_FILE
 
     val timestampFileType = FileFactory.getFileType(timestampFile)
     (timestampFile, timestampFileType)
@@ -627,14 +586,8 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
 
     touchSchemasTimestampFile(schemaName, cubeName)
 
-    if (useUniquePath) {
-      cubeModifiedTimeStore.put(schemaName + '_' + cubeName,
-        FileFactory.getCarbonFile(timestampFile, timestampFileType).getLastModifiedTime)
-    }
-    else {
-      cubeModifiedTimeStore.put("default",
-        FileFactory.getCarbonFile(timestampFile, timestampFileType).getLastModifiedTime)
-    }
+    cubeModifiedTimeStore.put("default",
+      FileFactory.getCarbonFile(timestampFile, timestampFileType).getLastModifiedTime)
 
   }
 
@@ -645,26 +598,11 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
   }
 
   def checkSchemasModifiedTimeAndReloadCubes() {
-    if (useUniquePath) {
-      metadata.cubesMeta.foreach(c => {
-        val (timestampFile, timestampFileType) = getTimestampFileAndType(
-          c.carbonTableIdentifier.getDatabaseName, c.carbonTableIdentifier.getTableName)
-
-        if (FileFactory.isFileExist(timestampFile, timestampFileType)) {
-          if (!(FileFactory.getCarbonFile(timestampFile, timestampFileType).getLastModifiedTime ==
-                cubeModifiedTimeStore.get(c.carbonTableIdentifier.getDatabaseName + "_" +
-                                          c.carbonTableIdentifier.getTableName))) {
-            refreshCache()
-          }
-        }
-      })
-    } else {
-      val (timestampFile, timestampFileType) = getTimestampFileAndType("", "")
-      if (FileFactory.isFileExist(timestampFile, timestampFileType)) {
-        if (!(FileFactory.getCarbonFile(timestampFile, timestampFileType).
-          getLastModifiedTime == cubeModifiedTimeStore.get("default"))) {
-          refreshCache()
-        }
+    val (timestampFile, timestampFileType) = getTimestampFileAndType("", "")
+    if (FileFactory.isFileExist(timestampFile, timestampFileType)) {
+      if (!(FileFactory.getCarbonFile(timestampFile, timestampFileType).
+        getLastModifiedTime == cubeModifiedTimeStore.get("default"))) {
+        refreshCache()
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/60490179/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 d77f8b5..4c4652c 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -86,12 +86,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(CarbonFactDataHandlerColumnar.class.getName());
   /**
-   * decimalPointers
-   */
-  private final byte decimalPointers = Byte.parseByte(CarbonProperties.getInstance()
-      .getProperty(CarbonCommonConstants.CARBON_DECIMAL_POINTERS,
-          CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT));
-  /**
    * data writer
    */
   private CarbonFactDataWriter dataWriter;
@@ -870,7 +864,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
           double minVal = (double) min[count];
           max[count] = (maxVal > value ? max[count] : value);
           min[count] = (minVal < value ? min[count] : value);
-          int num = (value % 1 == 0) ? 0 : decimalPointers;
+          int num = (value % 1 == 0) ? 0 : CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT;
           decimal[count] = (decimal[count] > num ? decimal[count] : num);
         } else if (type[count] == CarbonCommonConstants.BIG_INT_MEASURE) {
           long value = (long) row[count];
@@ -878,7 +872,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
           long minVal = (long) min[count];
           max[count] = (maxVal > value ? max[count] : value);
           min[count] = (minVal < value ? min[count] : value);
-          int num = (value % 1 == 0) ? 0 : decimalPointers;
+          int num = (value % 1 == 0) ? 0 : CarbonCommonConstants.CARBON_DECIMAL_POINTERS_DEFAULT;
           decimal[count] = (decimal[count] > num ? decimal[count] : num);
         } else if (type[count] == CarbonCommonConstants.BIG_DECIMAL_MEASURE) {
           byte[] buff = (byte[]) row[count];


[10/50] [abbrv] incubator-carbondata git commit: [BUG]Column group with no dictionary and timestamp (#713)

Posted by ch...@apache.org.
[BUG]Column group with no dictionary and timestamp (#713)

While data loading, encoding is not proper for all dimension

Fixed query execution for no dictionary and column groups
Fixed test cases and added more test case for column group and dictionary exclude
Fixed review comment

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

Branch: refs/heads/master
Commit: e96de9f5b9e0560a5e59cc5f941e450d4a7e00f8
Parents: 82332b0
Author: ashokblend <as...@gmail.com>
Authored: Sat Jun 25 14:05:56 2016 +0530
Committer: Kumar Vishal <ku...@gmail.com>
Committed: Sat Jun 25 16:35:56 2016 +0800

----------------------------------------------------------------------
 .../datastore/block/SegmentProperties.java      |  88 ++++-
 .../impl/ColumnGroupDimensionDataChunk.java     |   7 +-
 .../core/util/CarbonMetadataUtil.java           |  36 +-
 .../org/carbondata/core/util/CarbonUtil.java    |  49 ++-
 .../carbondata/core/vo/ColumnGroupModel.java    |  21 -
 .../carbon/executor/infos/KeyStructureInfo.java |  18 -
 .../query/carbon/executor/util/QueryUtil.java   |  35 +-
 .../executer/ColGroupFilterExecuterImpl.java    |  51 ++-
 .../core/writer/CarbonFooterWriterTest.java     |  58 ++-
 .../spark/merger/RowResultMerger.java           |  20 +-
 .../ColumnGroupDataTypesTestCase.scala          |  33 +-
 .../processing/mdkeygen/MDKeyGenStep.java       |  41 +-
 .../store/CarbonFactDataHandlerColumnar.java    | 101 ++---
 .../store/CarbonFactDataHandlerModel.java       |  63 ++-
 .../store/colgroup/ColGroupDataHolder.java      |   4 +-
 .../store/colgroup/ColGroupMinMax.java          |  30 +-
 .../store/writer/AbstractFactDataWriter.java    |   8 +-
 ...actDataWriterImplForIntIndexAndAggBlock.java |   6 +-
 .../store/colgroup/ColGroupMinMaxTest.java      | 380 ++++++++++---------
 19 files changed, 606 insertions(+), 443 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java b/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
index 137468d..f6f369a 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
@@ -21,10 +21,12 @@ package org.carbondata.core.carbon.datastore.block;
 import java.util.ArrayList;
 import java.util.Arrays;
 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 org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
@@ -37,6 +39,7 @@ import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
 import org.carbondata.core.keygenerator.mdkey.MultiDimKeyVarLengthGenerator;
 import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.core.vo.ColumnGroupModel;
 
 import org.apache.commons.lang3.ArrayUtils;
 
@@ -85,6 +88,12 @@ public class SegmentProperties {
   private Map<Integer, Integer> dimensionOrdinalToBlockMapping;
 
   /**
+   * a block can have multiple columns. This will have block index as key
+   * and all dimension participated in that block as values
+   */
+  private Map<Integer, Set<Integer>> blockTodimensionOrdinalMapping;
+
+  /**
    * mapping of measure column to block to in file this will be used while
    * reading the block in a file
    */
@@ -114,6 +123,12 @@ public class SegmentProperties {
   private Map<Integer, KeyGenerator> columnGroupAndItsKeygenartor;
 
   /**
+   * column group key generator dimension index will not be same as dimension ordinal
+   * This will have mapping with ordinal and keygenerator or mdkey index
+   */
+  private Map<Integer, Map<Integer, Integer>> columnGroupOrdinalToMdkeymapping;
+
+  /**
    * this will be used to split the fixed length key
    * this will all the information about how key was created
    * and how to split the key based on group
@@ -129,9 +144,9 @@ public class SegmentProperties {
   private int numberOfNoDictionaryDimension;
 
   /**
-   * column groups
+   * column group model
    */
-  private int[][] colGroups;
+  private ColumnGroupModel colGroupModel;
 
   public SegmentProperties(List<ColumnSchema> columnsInTable, int[] columnCardinality) {
     dimensions = new ArrayList<CarbonDimension>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -141,6 +156,8 @@ public class SegmentProperties {
     fillDimensionAndMeasureDetails(columnsInTable, columnCardinality);
     dimensionOrdinalToBlockMapping =
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    blockTodimensionOrdinalMapping =
+        new HashMap<Integer, Set<Integer>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     measuresOrdinalToBlockMapping =
         new HashMap<Integer, Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
     intialiseColGroups(columnsInTable);
@@ -179,13 +196,14 @@ public class SegmentProperties {
       }
 
     }
-    colGroups = new int[colGrpList.size()][];
+    int[][] colGroups = new int[colGrpList.size()][];
     for (int i = 0; i < colGroups.length; i++) {
       colGroups[i] = new int[colGrpList.get(i).size()];
       for (int j = 0; j < colGroups[i].length; j++) {
         colGroups[i][j] = colGrpList.get(i).get(j);
       }
     }
+    this.colGroupModel = CarbonUtil.getColGroupModel(colGroups);
   }
 
   /**
@@ -215,6 +233,24 @@ public class SegmentProperties {
       dimensionOrdinalToBlockMapping.put(dimension.getOrdinal(), ++blockOrdinal);
       index++;
     }
+    fillBlockToDimensionOrdinalMapping();
+  }
+
+  /**
+   *
+   */
+  private void fillBlockToDimensionOrdinalMapping() {
+    Set<Entry<Integer, Integer>> blocks = dimensionOrdinalToBlockMapping.entrySet();
+    Iterator<Entry<Integer, Integer>> blockItr = blocks.iterator();
+    while (blockItr.hasNext()) {
+      Entry<Integer, Integer> block = blockItr.next();
+      Set<Integer> dimensionOrdinals = blockTodimensionOrdinalMapping.get(block.getValue());
+      if (dimensionOrdinals == null) {
+        dimensionOrdinals = new HashSet<Integer>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+        blockTodimensionOrdinalMapping.put(block.getValue(), dimensionOrdinals);
+      }
+      dimensionOrdinals.add(block.getKey());
+    }
   }
 
   /**
@@ -457,6 +493,7 @@ public class SegmentProperties {
     // Initializing the map
     this.columnGroupAndItsKeygenartor =
         new HashMap<Integer, KeyGenerator>(columnGroupAndOrdinalMapping.size());
+    this.columnGroupOrdinalToMdkeymapping = new HashMap<>(columnGroupAndOrdinalMapping.size());
     int[] columnGroupCardinality = null;
     index = 0;
     Iterator<Entry<Integer, List<Integer>>> iterator =
@@ -464,14 +501,18 @@ public class SegmentProperties {
     while (iterator.hasNext()) {
       Entry<Integer, List<Integer>> next = iterator.next();
       List<Integer> currentGroupOrdinal = next.getValue();
+      Map<Integer, Integer> colGrpOrdinalMdkeyMapping = new HashMap<>(currentGroupOrdinal.size());
       // create the cardinality array
       columnGroupCardinality = new int[currentGroupOrdinal.size()];
       for (int i = 0; i < columnGroupCardinality.length; i++) {
         // fill the cardinality
         columnGroupCardinality[i] = cardinality[currentGroupOrdinal.get(i)];
+        colGrpOrdinalMdkeyMapping.put(currentGroupOrdinal.get(i), i);
       }
       this.columnGroupAndItsKeygenartor.put(next.getKey(), new MultiDimKeyVarLengthGenerator(
-          CarbonUtil.getDimensionBitLength(cardinality, new int[] { cardinality.length })));
+          CarbonUtil.getDimensionBitLength(columnGroupCardinality,
+              new int[] { columnGroupCardinality.length })));
+      this.columnGroupOrdinalToMdkeymapping.put(next.getKey(), colGrpOrdinalMdkeyMapping);
     }
   }
 
@@ -586,11 +627,46 @@ public class SegmentProperties {
   }
 
   /**
-   *
    * @return
    */
   public int[][] getColumnGroups() {
-    return this.colGroups;
+    return colGroupModel.getColumnGroup();
+  }
+
+  /**
+   * @return colGroupModel
+   */
+  public ColumnGroupModel getColumnGroupModel() {
+    return this.colGroupModel;
+  }
+
+  /**
+   * get mdkey ordinal for given dimension ordinal of given column group
+   *
+   * @param colGrpId
+   * @param ordinal
+   * @return mdkeyordinal
+   */
+  public int getColumnGroupMdKeyOrdinal(int colGrpId, int ordinal) {
+    return columnGroupOrdinalToMdkeymapping.get(colGrpId).get(ordinal);
+  }
+
+  /**
+   * It returns no of column availble in given column group
+   *
+   * @param colGrpId
+   * @return no of column in given column group
+   */
+  public int getNoOfColumnsInColumnGroup(int colGrpId) {
+    return columnGroupOrdinalToMdkeymapping.get(colGrpId).size();
+  }
+
+  /**
+   * @param blockIndex
+   * @return It returns all dimension present in given block index
+   */
+  public Set<Integer> getDimensionOrdinalForBlock(int blockIndex) {
+    return blockTodimensionOrdinalMapping.get(blockIndex);
   }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..7d0cc26 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
@@ -60,11 +60,8 @@ public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<b
    */
   @Override public int fillChunkData(byte[] data, int offset, int rowId,
       KeyStructureInfo restructuringInfo) {
-    byte[] rowData = new byte[restructuringInfo.getMaskedBytes().length];
-    System.arraycopy(dataChunk, rowId * chunkAttributes.getColumnValueSize(), rowData,
-        restructuringInfo.getBlockMdKeyStartOffset(), chunkAttributes.getColumnValueSize());
     byte[] maskedKey =
-        getMaskedKey(rowData, rowId * chunkAttributes.getColumnValueSize(), restructuringInfo);
+        getMaskedKey(dataChunk, rowId * chunkAttributes.getColumnValueSize(), restructuringInfo);
     System.arraycopy(maskedKey, 0, data, offset, maskedKey.length);
     return maskedKey.length;
   }
@@ -81,7 +78,7 @@ public class ColumnGroupDimensionDataChunk implements DimensionColumnDataChunk<b
     int byteRange = 0;
     for (int i = 0; i < info.getMaskByteRanges().length; i++) {
       byteRange = info.getMaskByteRanges()[i];
-      maskedKey[counter++] = (byte) (data[byteRange] & info.getMaxKey()[byteRange]);
+      maskedKey[counter++] = (byte) (data[byteRange + offset] & info.getMaxKey()[byteRange]);
     }
     return maskedKey;
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
index 5e8e8a2..9fbcc47 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
@@ -8,9 +8,11 @@ import java.io.ObjectOutputStream;
 import java.nio.ByteBuffer;
 import java.util.ArrayList;
 import java.util.List;
+import java.util.Set;
 
 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.metadata.index.BlockIndexInfo;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
@@ -52,7 +54,8 @@ public class CarbonMetadataUtil {
    * @return FileFooter
    */
   public static FileFooter convertFileFooter(List<BlockletInfoColumnar> infoList, int numCols,
-      int[] cardinalities, List<ColumnSchema> columnSchemaList) throws IOException {
+      int[] cardinalities, List<ColumnSchema> columnSchemaList,
+      SegmentProperties segmentProperties) throws IOException {
 
     SegmentInfo segmentInfo = new SegmentInfo();
     segmentInfo.setNum_cols(columnSchemaList.size());
@@ -66,7 +69,7 @@ public class CarbonMetadataUtil {
     }
     footer.setTable_columns(columnSchemaList);
     for (BlockletInfoColumnar info : infoList) {
-      footer.addToBlocklet_info_list(getBlockletInfo(info, columnSchemaList));
+      footer.addToBlocklet_info_list(getBlockletInfo(info, columnSchemaList, segmentProperties));
     }
     return footer;
   }
@@ -122,7 +125,8 @@ public class CarbonMetadataUtil {
   }
 
   private static BlockletInfo getBlockletInfo(BlockletInfoColumnar blockletInfoColumnar,
-      List<ColumnSchema> columnSchenma) throws IOException {
+      List<ColumnSchema> columnSchenma,
+      SegmentProperties segmentProperties) throws IOException {
 
     BlockletInfo blockletInfo = new BlockletInfo();
     blockletInfo.setNum_rows(blockletInfoColumnar.getNumberOfKeys());
@@ -138,10 +142,10 @@ public class CarbonMetadataUtil {
       DataChunk dataChunk = new DataChunk();
       dataChunk.setChunk_meta(getChunkCompressionMeta());
       List<Encoding> encodings = new ArrayList<Encoding>();
-      if (columnSchenma.get(i).encoders.contains(Encoding.DICTIONARY)) {
+      if (containsEncoding(i, Encoding.DICTIONARY, columnSchenma, segmentProperties)) {
         encodings.add(Encoding.DICTIONARY);
       }
-      if (columnSchenma.get(i).encoders.contains(Encoding.DIRECT_DICTIONARY)) {
+      if (containsEncoding(i, Encoding.DIRECT_DICTIONARY, columnSchenma, segmentProperties)) {
         encodings.add(Encoding.DIRECT_DICTIONARY);
       }
       dataChunk.setRowMajor(colGrpblock[i]);
@@ -204,6 +208,28 @@ public class CarbonMetadataUtil {
     return blockletInfo;
   }
 
+  /**
+   * @param blockIndex
+   * @param encoding
+   * @param columnSchemas
+   * @param segmentProperties
+   * @return return true if given encoding is present in column
+   */
+  private static boolean containsEncoding(int blockIndex, Encoding encoding,
+      List<ColumnSchema> columnSchemas, SegmentProperties segmentProperties) {
+    Set<Integer> dimOrdinals = segmentProperties.getDimensionOrdinalForBlock(blockIndex);
+    //column groups will always have dictionary encoding
+    if (dimOrdinals.size() > 1 && Encoding.DICTIONARY == encoding) {
+      return true;
+    }
+    for (Integer dimOrdinal : dimOrdinals) {
+      if (columnSchemas.get(dimOrdinal).encoders.contains(encoding)) {
+        return true;
+      }
+    }
+    return false;
+  }
+
   private static byte[] serializeEncoderMeta(ValueEncoderMeta encoderMeta) throws IOException {
     // TODO : should remove the unnecessary fields.
     ByteArrayOutputStream aos = new ByteArrayOutputStream();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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 427fb97..2d50f07 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -56,6 +56,8 @@ import org.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 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.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;
@@ -364,11 +366,10 @@ public final class CarbonUtil {
   /**
    * return ColumnGroupModel. check ColumnGroupModel for detail
    *
-   * @param dimLens      : dimension cardinality
    * @param columnGroups : column groups
    * @return ColumnGroupModel  model
    */
-  public static ColumnGroupModel getColGroupModel(int[] dimLens, int[][] columnGroups) {
+  public static ColumnGroupModel getColGroupModel(int[][] columnGroups) {
     int[] columnSplit = new int[columnGroups.length];
     int noOfColumnStore = columnSplit.length;
     boolean[] columnarStore = new boolean[noOfColumnStore];
@@ -380,7 +381,6 @@ public final class CarbonUtil {
     ColumnGroupModel colGroupModel = new ColumnGroupModel();
     colGroupModel.setNoOfColumnStore(noOfColumnStore);
     colGroupModel.setColumnSplit(columnSplit);
-    colGroupModel.setColumnGroupCardinality(dimLens);
     colGroupModel.setColumnarStore(columnarStore);
     colGroupModel.setColumnGroup(columnGroups);
     return colGroupModel;
@@ -1688,6 +1688,49 @@ public final class CarbonUtil {
   }
 
   /**
+   * @param dictionaryColumnCardinality
+   * @param wrapperColumnSchemaList
+   * @return It returns formatted cardinality by adding -1 value for NoDictionary columns
+   */
+  public static int[] getFormattedCardinality(int[] dictionaryColumnCardinality,
+      List<ColumnSchema> wrapperColumnSchemaList) {
+    List<Integer> cardinality = new ArrayList<>();
+    int counter = 0;
+    for (int i = 0; i < wrapperColumnSchemaList.size(); i++) {
+      if (CarbonUtil.hasEncoding(wrapperColumnSchemaList.get(i).getEncodingList(),
+          org.carbondata.core.carbon.metadata.encoder.Encoding.DICTIONARY)) {
+        cardinality.add(dictionaryColumnCardinality[counter]);
+        counter++;
+      } else if (!wrapperColumnSchemaList.get(i).isDimensionColumn()) {
+        continue;
+      } else {
+        cardinality.add(-1);
+      }
+    }
+    return ArrayUtils.toPrimitive(cardinality.toArray(new Integer[cardinality.size()]));
+  }
+
+  public static List<ColumnSchema> getColumnSchemaList(List<CarbonDimension> carbonDimensionsList,
+      List<CarbonMeasure> carbonMeasureList) {
+    List<ColumnSchema> wrapperColumnSchemaList = new ArrayList<ColumnSchema>();
+    fillCollumnSchemaListForComplexDims(carbonDimensionsList, wrapperColumnSchemaList);
+    for (CarbonMeasure carbonMeasure : carbonMeasureList) {
+      wrapperColumnSchemaList.add(carbonMeasure.getColumnSchema());
+    }
+    return wrapperColumnSchemaList;
+  }
+
+  private static void fillCollumnSchemaListForComplexDims(
+      List<CarbonDimension> carbonDimensionsList, List<ColumnSchema> wrapperColumnSchemaList) {
+    for (CarbonDimension carbonDimension : carbonDimensionsList) {
+      wrapperColumnSchemaList.add(carbonDimension.getColumnSchema());
+      List<CarbonDimension> childDims = carbonDimension.getListOfChildDimensions();
+      if (null != childDims && childDims.size() > 0) {
+        fillCollumnSchemaListForComplexDims(childDims, wrapperColumnSchemaList);
+      }
+    }
+  }
+  /**
    * Below method will be used to get all the block index info from index file
    *
    * @param taskId                  task id of the file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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
index b3b9d37..a778598 100644
--- a/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
+++ b/core/src/main/java/org/carbondata/core/vo/ColumnGroupModel.java
@@ -21,11 +21,6 @@ package org.carbondata.core.vo;
 public class ColumnGroupModel {
 
   /**
-   * cardinality all dimension
-   */
-  private int[] colGroupCardinality;
-
-  /**
    * number of columns in columnar block
    */
   private int[] columnSplit;
@@ -50,22 +45,6 @@ public class ColumnGroupModel {
   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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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
index 75e65a2..841184b 100644
--- 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
@@ -49,11 +49,6 @@ public class KeyStructureInfo {
   private byte[] maxKey;
 
   /**
-   * mdkey start index of block
-   */
-  private int blockMdKeyStartOffset;
-
-  /**
    * @return the keyGenerator
    */
   public KeyGenerator getKeyGenerator() {
@@ -109,17 +104,4 @@ public class KeyStructureInfo {
     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/e96de9f5/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 a8eaa49..1d68872 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
@@ -528,29 +528,33 @@ public class QueryUtil {
     // get column group id and its ordinal mapping of column group
     Map<Integer, List<Integer>> columnGroupAndItsOrdinalMappingForQuery =
         getColumnGroupAndItsOrdinalMapping(queryDimensions);
-    KeyGenerator keyGenerator = segmentProperties.getDimensionKeyGenerator();
+    Map<Integer, KeyGenerator> columnGroupAndItsKeygenartor =
+        segmentProperties.getColumnGroupAndItsKeygenartor();
 
     Iterator<Entry<Integer, List<Integer>>> iterator =
         columnGroupAndItsOrdinalMappingForQuery.entrySet().iterator();
     KeyStructureInfo restructureInfos = null;
     while (iterator.hasNext()) {
       Entry<Integer, List<Integer>> next = iterator.next();
+      KeyGenerator keyGenerator = columnGroupAndItsKeygenartor.get(next.getKey());
       restructureInfos = new KeyStructureInfo();
       // sort the ordinal
       List<Integer> ordinal = next.getValue();
-      Collections.sort(ordinal);
+      List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+      for (Integer ord : ordinal) {
+        mdKeyOrdinal.add(segmentProperties.getColumnGroupMdKeyOrdinal(next.getKey(), ord));
+      }
+      Collections.sort(mdKeyOrdinal);
       // get the masked byte range for column group
-      int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(ordinal, keyGenerator);
+      int[] maskByteRanges = getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
       // max key for column group
-      byte[] maxKey = getMaxKeyBasedOnOrinal(ordinal, keyGenerator);
+      byte[] maxKey = getMaxKeyBasedOnOrinal(mdKeyOrdinal, 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);
@@ -559,25 +563,6 @@ public class QueryUtil {
   }
 
   /**
-   * 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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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
index e4e3812..98b3ed4 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
@@ -27,6 +27,7 @@ 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.keygenerator.KeyGenerator;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.query.carbon.executor.infos.KeyStructureInfo;
 import org.carbondata.query.carbon.executor.util.QueryUtil;
@@ -88,24 +89,18 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
    * @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);
-
+    int colGrpId = getColumnGroupId(dimColumnEvaluatorInfo.getColumnIndex());
+    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+    mdKeyOrdinal.add(getMdkeyOrdinal(dimColumnEvaluatorInfo.getColumnIndex(), colGrpId));
+    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
+    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
+    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
     KeyStructureInfo restructureInfos = new KeyStructureInfo();
-    restructureInfos.setKeyGenerator(segmentProperties.getDimensionKeyGenerator());
+    restructureInfos.setKeyGenerator(keyGenerator);
     restructureInfos.setMaskByteRanges(maskByteRanges);
     restructureInfos.setMaxKey(maxKey);
     restructureInfos.setMaskedBytes(maksedByte);
-    restructureInfos.setBlockMdKeyStartOffset(blockMdkeyStartOffset);
     return restructureInfos;
   }
 
@@ -155,8 +150,9 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
     int endIndex = 0;
     if (null != colGrpColumns) {
       for (int i = 0; i < colGrpColumns.length; i++) {
-        int[] byteRange =
-            segmentProperties.getDimensionKeyGenerator().getKeyByteOffsets(colGrpColumns[i]);
+        int colGrpId = getColumnGroupId(colGrpColumns[i]);
+        int mdKeyOrdinal = getMdkeyOrdinal(colGrpColumns[i], colGrpId);
+        int[] byteRange = getKeyGenerator(colGrpId).getKeyByteOffsets(mdKeyOrdinal);
         int colSize = 0;
         for (int j = byteRange[0]; j <= byteRange[1]; j++) {
           colSize++;
@@ -189,4 +185,25 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
     return null;
   }
 
-}
+  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
+    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
+  }
+
+  private int getColumnGroupId(int ordinal) {
+    int[][] columnGroups = segmentProperties.getColumnGroups();
+    int colGrpId = -1;
+    for (int i = 0; i < columnGroups.length; i++) {
+      if (columnGroups[i].length > 1) {
+        colGrpId++;
+        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
+          break;
+        }
+      }
+    }
+    return colGrpId;
+  }
+
+  public KeyGenerator getKeyGenerator(int colGrpId) {
+    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java b/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
index 14b019c..bfcbdde 100644
--- a/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
+++ b/core/src/test/java/org/carbondata/core/writer/CarbonFooterWriterTest.java
@@ -28,6 +28,9 @@ import java.util.UUID;
 
 import junit.framework.TestCase;
 
+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.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
@@ -35,6 +38,7 @@ import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.metadata.BlockletInfoColumnar;
 import org.carbondata.core.reader.CarbonFooterReader;
 import org.carbondata.core.util.CarbonMetadataUtil;
+import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.format.ColumnSchema;
 import org.junit.After;
 import org.junit.Before;
@@ -68,16 +72,26 @@ public class CarbonFooterWriterTest extends TestCase{
 
     List<BlockletInfoColumnar> infoColumnars = getBlockletInfoColumnars();
 
-		writer.writeFooter(CarbonMetadataUtil.convertFileFooter(
-				infoColumnars,
-				6,
-				new int[] { 2, 4, 5, 7 },
-				Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
+    int[] cardinalities = new int[] { 2, 4, 5, 7, 9, 10 };
+    List<ColumnSchema> columnSchema = Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
 						getDimensionColumn("IMEI2"),
 						getDimensionColumn("IMEI3"),
 						getDimensionColumn("IMEI4"),
 						getDimensionColumn("IMEI5"),
-						getDimensionColumn("IMEI6")})), 0);
+						getDimensionColumn("IMEI6")});
+    List<org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema> wrapperColumnSchema = Arrays.asList(new org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema[]{getWrapperDimensionColumn("IMEI1"),
+    	getWrapperDimensionColumn("IMEI2"),
+    	getWrapperDimensionColumn("IMEI3"),
+    	getWrapperDimensionColumn("IMEI4"),
+    	getWrapperDimensionColumn("IMEI5"),
+    	getWrapperDimensionColumn("IMEI6")});
+    int[] colCardinality = CarbonUtil.getFormattedCardinality(cardinalities, wrapperColumnSchema);
+    SegmentProperties segmentProperties = new SegmentProperties(wrapperColumnSchema, colCardinality);
+		writer.writeFooter(CarbonMetadataUtil.convertFileFooter(
+				infoColumnars,
+				6,
+				cardinalities,columnSchema, segmentProperties
+				), 0);
 
     CarbonFooterReader metaDataReader = new CarbonFooterReader(filePath, 0);
     assertTrue(metaDataReader.readFooter() != null);
@@ -97,6 +111,20 @@ public class CarbonFooterWriterTest extends TestCase{
 	    dimColumn.setNum_child(0);
 	    return dimColumn;
 	  }
+  public static org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema getWrapperDimensionColumn(String columnName) {
+   org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema dimColumn = new org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema();
+   dimColumn.setColumnar(true);
+   dimColumn.setColumnName(columnName);
+   dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
+   dimColumn.setDataType(DataType.STRING);
+   dimColumn.setDimensionColumn(true);
+   List<Encoding> encodeList =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+   encodeList.add(Encoding.DICTIONARY);
+   dimColumn.setEncodingList(encodeList);
+   dimColumn.setNumberOfChild(0);;
+   return dimColumn;
+ }
 
   /**
    * test writing fact metadata.
@@ -106,14 +134,24 @@ public class CarbonFooterWriterTest extends TestCase{
     createFile();
     CarbonFooterWriter writer = new CarbonFooterWriter(filePath);
     List<BlockletInfoColumnar> infoColumnars = getBlockletInfoColumnars();
-    writer.writeFooter(CarbonMetadataUtil
-        .convertFileFooter(infoColumnars, 6, new int[] { 2, 4, 5, 7 },
-        		Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
+    int[] cardinalities = new int[] { 2, 4, 5, 7, 9, 10};
+    List<ColumnSchema> columnSchema = Arrays.asList(new ColumnSchema[]{getDimensionColumn("IMEI1"),
 						getDimensionColumn("IMEI2"),
 						getDimensionColumn("IMEI3"),
 						getDimensionColumn("IMEI4"),
 						getDimensionColumn("IMEI5"),
-						getDimensionColumn("IMEI6")})), 0);
+						getDimensionColumn("IMEI6")});
+    List<org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema> wrapperColumnSchema = Arrays.asList(new org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema[]{getWrapperDimensionColumn("IMEI1"),
+    	getWrapperDimensionColumn("IMEI2"),
+    	getWrapperDimensionColumn("IMEI3"),
+    	getWrapperDimensionColumn("IMEI4"),
+    	getWrapperDimensionColumn("IMEI5"),
+    	getWrapperDimensionColumn("IMEI6")});
+    int[] colCardinality = CarbonUtil.getFormattedCardinality(cardinalities, wrapperColumnSchema);
+    SegmentProperties segmentProperties = new SegmentProperties(wrapperColumnSchema, cardinalities);
+    writer.writeFooter(CarbonMetadataUtil
+        .convertFileFooter(infoColumnars, 6, colCardinality,
+        		columnSchema,segmentProperties), 0);
 
     CarbonFooterReader metaDataReader = new CarbonFooterReader(filePath, 0);
     List<BlockletInfoColumnar> nodeInfoColumnars =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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 dcdc7a4..71117de 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
@@ -35,6 +35,7 @@ import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+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;
@@ -42,7 +43,6 @@ 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;
@@ -74,15 +74,13 @@ public class RowResultMerger {
   private AbstractQueue<RawResultIterator> recordHolderHeap;
 
   private TupleConversionAdapter tupleConvertor;
-  private ColumnGroupModel colGrpStoreModel;
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(RowResultMerger.class.getName());
 
-  public RowResultMerger(List<RawResultIterator> iteratorList,
-      String schemaName, String tableName,
-      SegmentProperties segProp, String tempStoreLocation,
-      CarbonLoadModel loadModel, int[] colCardinality) {
+  public RowResultMerger(List<RawResultIterator> iteratorList, String schemaName, String tableName,
+      SegmentProperties segProp, String tempStoreLocation, CarbonLoadModel loadModel,
+      int[] colCardinality) {
 
     this.rawResultIteratorList = iteratorList;
     // create the List of RawResultIterator.
@@ -222,16 +220,20 @@ public class RowResultMerger {
         .setMdKeyLength(segprop.getDimensionKeyGenerator().getKeySizeInBytes());
     carbonFactDataHandlerModel.setStoreLocation(tempStoreLocation);
     carbonFactDataHandlerModel.setDimLens(segprop.getDimColumnsCardinality());
+    carbonFactDataHandlerModel.setSegmentProperties(segprop);
     carbonFactDataHandlerModel.setNoDictionaryCount(segprop.getNumberOfNoDictionaryDimension());
     carbonFactDataHandlerModel.setDimensionCount(
         segprop.getDimensions().size() - carbonFactDataHandlerModel.getNoDictionaryCount());
+    CarbonTable carbonTable = CarbonMetadata.getInstance()
+        .getCarbonTable(schemaName + CarbonCommonConstants.UNDERSCORE + tableName);
+    List<ColumnSchema> wrapperColumnSchema = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
+            carbonTable.getMeasureByTableName(tableName));
+    carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema);
     //TO-DO Need to handle complex types here .
     Map<Integer, GenericDataType> complexIndexMap =
         new HashMap<Integer, GenericDataType>(segprop.getComplexDimensions().size());
     carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap);
-    this.colGrpStoreModel =
-        CarbonUtil.getColGroupModel(segprop.getDimColumnsCardinality(), segprop.getColumnGroups());
-    carbonFactDataHandlerModel.setColGrpModel(colGrpStoreModel);
     carbonFactDataHandlerModel.setDataWritingRequest(true);
 
     char[] aggType = new char[segprop.getMeasures().size()];

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
index 8371018..57f296a 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
@@ -23,7 +23,6 @@ import org.apache.spark.sql.common.util.CarbonHiveContext._
 import org.apache.spark.sql.common.util.QueryTest
 import org.scalatest.BeforeAndAfterAll
 
-
 /**
  * Test Class for aggregate query on multiple datatypes
  *
@@ -35,6 +34,12 @@ class ColumnGroupDataTypesTestCase extends QueryTest with BeforeAndAfterAll {
     sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table colgrp options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
     sql("create table normal (column1 string,column2 string,column3 string,column4 string,column5 string,column6 string,column7 string,column8 string,column9 string,column10 string,measure1 int,measure2 int,measure3 int,measure4 int) STORED BY 'org.apache.carbondata.format'")
     sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table normal options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
+    //column group with dictionary exclude before column group
+    sql("create table colgrp_dictexclude_before (column1 string,column2 string,column3 string,column4 string,column5 string,column6 string,column7 string,column8 string,column9 string,column10 string,measure1 int,measure2 int,measure3 int,measure4 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_EXCLUDE'='column1',\"COLUMN_GROUPS\"=\"(column2,column3,column4),(column7,column8,column9)\")")
+    sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table colgrp_dictexclude_before options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
+    //column group with dictionary exclude after column group
+    sql("create table colgrp_dictexclude_after (column1 string,column2 string,column3 string,column4 string,column5 string,column6 string,column7 string,column8 string,column9 string,column10 string,measure1 int,measure2 int,measure3 int,measure4 int) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES ('DICTIONARY_EXCLUDE'='column10',\"COLUMN_GROUPS\"=\"(column2,column3,column4),(column7,column8,column9)\")")
+    sql("LOAD DATA LOCAL INPATH './src/test/resources/10dim_4msr.csv' INTO table colgrp_dictexclude_after options('FILEHEADER'='column1,column2,column3,column4,column5,column6,column7,column8,column9,column10,measure1,measure2,measure3,measure4')");
   }
 
   test("select all dimension query") {
@@ -79,8 +84,30 @@ class ColumnGroupDataTypesTestCase extends QueryTest with BeforeAndAfterAll {
       sql("select column1,column3,column4,column5,column6,column9,column10 from normal"))
   }
 
+   test("##ColumnGroup_DictionaryExcludeBefore select all dimension on column group and dictionary exclude table") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_before"),
+      sql("select * from normal"))
+  }
+  test("##ColumnGroup_DictionaryExcludeBefore select all dimension query with filter on two dimension from same column group") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_before where column3='column311' and column4='column42' "),
+      sql("select * from normal where column3='column311' and column4='column42'"))
+  }
+  test("##ColumnGroup_DictionaryExcludeAfter select all dimension on column group and dictionary exclude table") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_after"),
+      sql("select * from normal"))
+  }
+  test("##ColumnGroup_DictionaryExcludeAfter select all dimension query with filter on two dimension from same column group") {
+    checkAnswer(
+      sql("select * from colgrp_dictexclude_after where column3='column311' and column4='column42' "),
+      sql("select * from normal where column3='column311' and column4='column42'"))
+  }
   override def afterAll {
-    sql("drop cube colgrp")
-    sql("drop cube normal")
+    sql("drop table colgrp")
+    sql("drop table normal")
+    sql("drop table colgrp_dictexclude_before")
+    sql("drop table colgrp_dictexclude_after")
   }
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..473840b 100644
--- a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@ -31,9 +31,11 @@ import java.util.Map.Entry;
 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.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+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;
@@ -41,12 +43,10 @@ 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.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.store.CarbonDataFileAttributes;
 import org.carbondata.processing.store.CarbonFactDataHandlerColumnar;
@@ -99,6 +99,12 @@ public class MDKeyGenStep extends BaseStep {
 
   private Map<Integer, GenericDataType> complexIndexMap;
 
+  private SegmentProperties segmentProperties;
+
+  private int[] colCardinality;
+
+  private List<ColumnSchema> wrapperColumnSchema;
+
   /**
    * readCounter
    */
@@ -126,7 +132,6 @@ public class MDKeyGenStep extends BaseStep {
 
   private int[] dimLens;
 
-  private ColumnGroupModel colGrpStoreModel;
   /**
    * to check whether dimension is of dictionary type
    * or not
@@ -287,23 +292,17 @@ public class MDKeyGenStep extends BaseStep {
       simpleDimsLen[i] = dimLens[i];
     }
 
-    String[] colStore = null != meta.getColumnGroupsString() ?
-        meta.getColumnGroupsString().split(",") :
-        new String[0];
-    int[][] colGroups = new int[colStore.length][];
-    for (int i = 0; i < colGroups.length; i++) {
-      String[] group = colStore[i].split("~");
-      colGroups[i] = new int[group.length];
-      for (int j = 0; j < colGroups[i].length; j++) {
-        colGroups[i][j] = Integer.parseInt(group[j]);
-      }
-    }
+    CarbonTable carbonTable = CarbonMetadata.getInstance()
+        .getCarbonTable(meta.getSchemaName() + CarbonCommonConstants.UNDERSCORE + tableName);
+    wrapperColumnSchema = CarbonUtil
+        .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
+            carbonTable.getMeasureByTableName(tableName));
+    colCardinality =
+        CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchema);
+    segmentProperties = new SegmentProperties(wrapperColumnSchema, colCardinality);
     // Actual primitive dimension used to generate start & end key
 
-    this.colGrpStoreModel = CarbonUtil.getColGroupModel(simpleDimsLen, colGroups);
-    data.generator = KeyGeneratorFactory
-        .getKeyGenerator(colGrpStoreModel.getColumnGroupCardinality(),
-            colGrpStoreModel.getColumnSplit());
+    data.generator = segmentProperties.getDimensionKeyGenerator();
 
     //To Set MDKey Index of each primitive type in complex type
     int surrIndex = simpleDimsCount;
@@ -377,10 +376,12 @@ public class MDKeyGenStep extends BaseStep {
     carbonFactDataHandlerModel.setNoDictionaryCount(meta.getNoDictionaryCount());
     carbonFactDataHandlerModel.setDimensionCount(dimensionCount);
     carbonFactDataHandlerModel.setComplexIndexMap(complexIndexMap);
-    carbonFactDataHandlerModel.setColGrpModel(colGrpStoreModel);
+    carbonFactDataHandlerModel.setSegmentProperties(segmentProperties);
+    carbonFactDataHandlerModel.setColCardinality(colCardinality);
     carbonFactDataHandlerModel.setDataWritingRequest(true);
     carbonFactDataHandlerModel.setAggType(aggType);
     carbonFactDataHandlerModel.setFactDimLens(dimLens);
+    carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema);
     return carbonFactDataHandlerModel;
   }
 
@@ -449,7 +450,7 @@ public class MDKeyGenStep extends BaseStep {
     }
     outputRow[l] = RemoveDictionaryUtil.getByteArrayForNoDictionaryCols(row);
 
-    int[] highCardExcludedRows = new int[colGrpStoreModel.getColumnGroupCardinality().length];
+    int[] highCardExcludedRows = new int[segmentProperties.getDimColumnsCardinality().length];
     for (int i = 0; i < highCardExcludedRows.length; i++) {
       Object key = RemoveDictionaryUtil.getDimension(i, row);
       highCardExcludedRows[i] = (Integer) key;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..d77f8b5 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
@@ -41,10 +41,9 @@ import java.util.concurrent.atomic.AtomicInteger;
 
 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.metadata.CarbonMetadata;
 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.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.columnar.BlockIndexerStorageForInt;
@@ -58,7 +57,6 @@ import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.keygenerator.KeyGenerator;
 import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
 import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthEquiSplitGenerator;
-import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;
 import org.carbondata.core.keygenerator.factory.KeyGeneratorFactory;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
@@ -262,6 +260,11 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   private int[] colCardinality;
 
   /**
+   * Segment properties
+   */
+  private SegmentProperties segmentProperties;
+
+  /**
    * CarbonFactDataHandler constructor
    */
   public CarbonFactDataHandlerColumnar(CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
@@ -324,24 +327,24 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   private void initParameters(CarbonFactDataHandlerModel carbonFactDataHandlerModel) {
     this.databaseName = carbonFactDataHandlerModel.getDatabaseName();
     this.tableName = carbonFactDataHandlerModel.getTableName();
+    this.type = carbonFactDataHandlerModel.getAggType();
+    this.segmentProperties = carbonFactDataHandlerModel.getSegmentProperties();
+    this.wrapperColumnSchemaList = carbonFactDataHandlerModel.getWrapperColumnSchema();
+    this.colCardinality = carbonFactDataHandlerModel.getColCardinality();
     this.storeLocation = carbonFactDataHandlerModel.getStoreLocation();
     this.measureCount = carbonFactDataHandlerModel.getMeasureCount();
     this.mdkeyLength = carbonFactDataHandlerModel.getMdKeyLength();
     this.mdKeyIndex = carbonFactDataHandlerModel.getMdKeyIndex();
     this.noDictionaryCount = carbonFactDataHandlerModel.getNoDictionaryCount();
-    this.colGrpModel = carbonFactDataHandlerModel.getColGrpModel();
+    this.colGrpModel = segmentProperties.getColumnGroupModel();
     this.completeDimLens = carbonFactDataHandlerModel.getDimLens();
-    this.dimLens = colGrpModel.getColumnGroupCardinality();
+    this.dimLens = this.segmentProperties.getDimColumnsCardinality();
     this.carbonDataFileAttributes = carbonFactDataHandlerModel.getCarbonDataFileAttributes();
-    this.type = carbonFactDataHandlerModel.getAggType();
     //TODO need to pass carbon table identifier to metadata
     CarbonTable carbonTable = CarbonMetadata.getInstance()
         .getCarbonTable(databaseName + CarbonCommonConstants.UNDERSCORE + tableName);
-    fillColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
-        carbonTable.getMeasureByTableName(tableName));
     dimensionType =
         CarbonUtil.identifyDimensionType(carbonTable.getDimensionByTableName(tableName));
-    this.colCardinality = carbonFactDataHandlerModel.getColCardinality();
 
     if (carbonFactDataHandlerModel.isCompactionFlow()) {
       try {
@@ -393,26 +396,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     return uniqueBlock;
   }
 
-  private void fillColumnSchemaList(List<CarbonDimension> carbonDimensionsList,
-      List<CarbonMeasure> carbonMeasureList) {
-    wrapperColumnSchemaList = new ArrayList<ColumnSchema>();
-    fillCollumnSchemaListForComplexDims(carbonDimensionsList, wrapperColumnSchemaList);
-    for (CarbonMeasure carbonMeasure : carbonMeasureList) {
-      wrapperColumnSchemaList.add(carbonMeasure.getColumnSchema());
-    }
-  }
-
-  private void fillCollumnSchemaListForComplexDims(List<CarbonDimension> carbonDimensionsList,
-      List<ColumnSchema> wrapperColumnSchemaList) {
-    for (CarbonDimension carbonDimension : carbonDimensionsList) {
-      wrapperColumnSchemaList.add(carbonDimension.getColumnSchema());
-      List<CarbonDimension> childDims = carbonDimension.getListOfChildDimensions();
-      if (null != childDims && childDims.size() > 0) {
-        fillCollumnSchemaListForComplexDims(childDims, wrapperColumnSchemaList);
-      }
-    }
-  }
-
   private void setComplexMapSurrogateIndex(int dimensionCount) {
     int surrIndex = 0;
     for (int i = 0; i < dimensionCount; i++) {
@@ -518,10 +501,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     if ((noDictionaryCount + complexColCount) > 0) {
       noDictionaryKeyDataHolder = initialiseKeyBlockHolder(dataRows.size());
     }
-    /**
-     * It holds min max value of columns of column group
-     */
-    ColGroupMinMax[] colGrpMinMax = initializeColGrpMinMax();
+
     for (int count = 0; count < dataRows.size(); count++) {
       Object[] row = dataRows.get(count);
       byte[] mdKey = (byte[]) row[this.mdKeyIndex];
@@ -529,8 +509,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       if (noDictionaryCount > 0 || complexIndexMap.size() > 0) {
         noDictionaryKey = (byte[]) row[this.mdKeyIndex - 1];
       }
-      //to evaluate min max value of columns in column group
-      evaluateColGrpMinMax(colGrpMinMax, mdKey);
       ByteBuffer byteBuffer = null;
       byte[] b = null;
       if (count == 0) {
@@ -601,8 +579,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
             .clone();
     NodeHolder nodeHolder =
         getNodeHolderObject(writableMeasureDataArray, byteArrayValues, dataRows.size(), startKey,
-            endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey,
-            colGrpMinMax);
+            endKey, compressionModel, noDictionaryValueHolder, noDictStartKey, noDictEndKey);
     nodeHolder.setMeasureNullValueIndex(nullValueIndexBitSet);
     LOGGER.info("Number Of records processed: " + dataRows.size());
     return nodeHolder;
@@ -611,7 +588,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   private NodeHolder getNodeHolderObject(byte[][] dataHolderLocal, byte[][] byteArrayValues,
       int entryCountLocal, byte[] startkeyLocal, byte[] endKeyLocal,
       ValueCompressionModel compressionModel, byte[][] noDictionaryData,
-      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey, ColGroupMinMax[] colGrpMinMax)
+      byte[] noDictionaryStartKey, byte[] noDictionaryEndKey)
       throws CarbonDataWriterException {
     byte[][][] noDictionaryColumnsData = null;
     List<ArrayList<byte[]>> colsAndValues = new ArrayList<ArrayList<byte[]>>();
@@ -621,7 +598,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       colsAndValues.add(new ArrayList<byte[]>());
     }
     int noOfColumn = colGrpModel.getNoOfColumnStore();
-    DataHolder[] dataHolders = getDataHolders(noOfColumn, byteArrayValues.length, colGrpMinMax);
+    DataHolder[] dataHolders = getDataHolders(noOfColumn, byteArrayValues.length);
     for (int i = 0; i < byteArrayValues.length; i++) {
       byte[][] splitKey = columnarSplitter.splitKey(byteArrayValues[i]);
 
@@ -741,14 +718,17 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
    * @param noOfRow    : total no of row
    * @return : dataholder
    */
-  private DataHolder[] getDataHolders(int noOfColumn, int noOfRow, ColGroupMinMax[] colGrpMinMax) {
+  private DataHolder[] getDataHolders(int noOfColumn, int noOfRow) {
     DataHolder[] dataHolders = new DataHolder[noOfColumn];
+    int colGrpId = -1;
     for (int colGrp = 0; colGrp < noOfColumn; colGrp++) {
       if (colGrpModel.isColumnar(colGrp)) {
         dataHolders[colGrp] = new ColumnDataHolder(noOfRow);
       } else {
-        dataHolders[colGrp] = new ColGroupDataHolder(this.colGrpModel,
-            this.columnarSplitter.getBlockKeySize()[colGrp], noOfRow, colGrpMinMax[colGrp]);
+        ColGroupMinMax colGrpMinMax = new ColGroupMinMax(segmentProperties, ++colGrpId);
+        dataHolders[colGrp] =
+            new ColGroupDataHolder(this.columnarSplitter.getBlockKeySize()[colGrp], noOfRow,
+                colGrpMinMax);
       }
     }
     return dataHolders;
@@ -876,34 +856,6 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
   }
 
   /**
-   * initialize column group min max evaluator
-   */
-  private ColGroupMinMax[] initializeColGrpMinMax() {
-    int[][] colGrps = colGrpModel.getColumnGroup();
-    ColGroupMinMax[] colGrpMinMax = new ColGroupMinMax[colGrps.length];
-    for (int colGrp = 0; colGrp < colGrps.length; colGrp++) {
-      if (!colGrpModel.isColumnar(colGrp)) {
-        colGrpMinMax[colGrp] = new ColGroupMinMax(colGrpModel, columnarSplitter, colGrp);
-      }
-    }
-    return colGrpMinMax;
-  }
-
-  /**
-   * Evaluate min max of columns in columnn group
-   *
-   * @param mdkey -> mdkey of data
-   */
-  private void evaluateColGrpMinMax(ColGroupMinMax[] colGrpMinMax, byte[] mdkey) {
-
-    for (int colGrp = 0; colGrp < colGrpModel.getColumnGroup().length; colGrp++) {
-      if (!colGrpModel.isColumnar(colGrp)) {
-        colGrpMinMax[colGrp].add(mdkey);
-      }
-    }
-  }
-
-  /**
    * This method will be used to update the max value for each measure
    */
   private void calculateMaxMin(Object[] max, Object[] min, int[] decimal, int[] msrIndex,
@@ -983,9 +935,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
       //row store will be in single column store
       //e.g if {0,1,2,3,4,5} is dimension and {0,1,2) is row store dimension
       //than below splitter will return column as {0,1,2}{3}{4}{5}
-      this.columnarSplitter = new MultiDimKeyVarLengthVariableSplitGenerator(CarbonUtil
-          .getDimensionBitLength(colGrpModel.getColumnGroupCardinality(),
-              colGrpModel.getColumnSplit()), colGrpModel.getColumnSplit());
+      this.columnarSplitter = this.segmentProperties.getFixedLengthKeySplitter();
       System.arraycopy(columnarSplitter.getBlockKeySize(), 0, keyBlockSize, 0, noOfColStore);
       this.keyBlockHolder =
           new CarbonKeyBlockHolder[this.columnarSplitter.getBlockKeySize().length];
@@ -1036,7 +986,7 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     this.dataWriter.setIsNoDictionary(isNoDictionary);
     // initialize the channel;
     this.dataWriter.initializeWriter();
-    initializeColGrpMinMax();
+    //initializeColGrpMinMax();
   }
 
   /**
@@ -1113,7 +1063,8 @@ public class CarbonFactDataHandlerColumnar implements CarbonFactHandler {
     return new CarbonFactDataWriterImplForIntIndexAndAggBlock(storeLocation, measureCount,
         mdKeyLength, tableName, fileManager, keyBlockSize, aggKeyBlock, isComplexTypes(),
         noDictionaryCount, carbonDataFileAttributes, databaseName, wrapperColumnSchemaList,
-        noDictionaryCount, dimensionType, carbonDataDirectoryPath, colCardinality);
+        noDictionaryCount, dimensionType, carbonDataDirectoryPath, colCardinality,
+        segmentProperties);
   }
 
   private boolean[] isComplexTypes() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..324d5fe 100644
--- a/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
+++ b/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
@@ -19,9 +19,11 @@
 
 package org.carbondata.processing.store;
 
+import java.util.List;
 import java.util.Map;
 
-import org.carbondata.core.vo.ColumnGroupModel;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.processing.datatypes.GenericDataType;
 
 /**
@@ -105,14 +107,17 @@ public class CarbonFactDataHandlerModel {
    * map which maintains indexing of complex columns
    */
   private Map<Integer, GenericDataType> complexIndexMap;
+
   /**
-   * primitive dimensions cardinality
+   * Segment properties
    */
-  private int[] primitiveDimLens;
+  private SegmentProperties segmentProperties;
+
   /**
-   * column group model
+   * primitive dimensions cardinality
    */
-  private ColumnGroupModel colGrpModel;
+  private int[] primitiveDimLens;
+
   /**
    * array in which each character represents an aggregation type and
    * the array length will be equal to the number of measures in table
@@ -127,9 +132,18 @@ public class CarbonFactDataHandlerModel {
    */
   private String carbonDataDirectoryPath;
 
+  /**
+   * cardinality of dimension including no dictionary. no dictionary cardinality
+   * is set to -1
+   */
   private int[] colCardinality;
 
   /**
+   * wrapper column schema
+   */
+  private List<ColumnSchema> wrapperColumnSchema;
+
+  /**
    * This is the boolean which will determine whether the data handler call is from the compaction
    * or not.
    */
@@ -302,14 +316,6 @@ public class CarbonFactDataHandlerModel {
     this.primitiveDimLens = primitiveDimLens;
   }
 
-  public ColumnGroupModel getColGrpModel() {
-    return colGrpModel;
-  }
-
-  public void setColGrpModel(ColumnGroupModel colGrpModel) {
-    this.colGrpModel = colGrpModel;
-  }
-
   public char[] getAggType() {
     return aggType;
   }
@@ -341,5 +347,36 @@ public class CarbonFactDataHandlerModel {
   public void setCompactionFlow(boolean compactionFlow) {
     isCompactionFlow = compactionFlow;
   }
+
+  /**
+   *
+   * @return segmentProperties
+   */
+  public SegmentProperties getSegmentProperties() {
+    return segmentProperties;
+  }
+
+  /**
+   *
+   * @param segmentProperties
+   */
+  public void setSegmentProperties(SegmentProperties segmentProperties) {
+    this.segmentProperties = segmentProperties;
+  }
+
+  /**
+   * @return wrapperColumnSchema
+   */
+  public List<ColumnSchema> getWrapperColumnSchema() {
+    return wrapperColumnSchema;
+  }
+
+  /**
+   * @param wrapperColumnSchema
+   */
+  public void setWrapperColumnSchema(List<ColumnSchema> wrapperColumnSchema) {
+    this.wrapperColumnSchema = wrapperColumnSchema;
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..9907ea2 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,6 @@
  */
 package org.carbondata.processing.store.colgroup;
 
-import org.carbondata.core.vo.ColumnGroupModel;
 
 /**
  * This will hold column group data.
@@ -48,7 +47,7 @@ public class ColGroupDataHolder implements DataHolder {
    * @param colGroupId
    * @param noOfRecords
    */
-  public ColGroupDataHolder(ColumnGroupModel colGrpModel, int keyBlockSize,
+  public ColGroupDataHolder(int keyBlockSize,
        int noOfRecords,ColGroupMinMax colGrpMinMax) {
     this.noOfRecords = noOfRecords;
     this.keyBlockSize = keyBlockSize;
@@ -58,6 +57,7 @@ public class ColGroupDataHolder implements DataHolder {
 
   @Override public void addData(byte[] rowsData, int rowIndex) {
     colGrpData[rowIndex] = rowsData;
+    colGrpMinMax.add(rowsData);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..768a128 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,10 @@ import java.util.Set;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 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
@@ -38,10 +37,6 @@ public class ColGroupMinMax {
 
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(ColGroupMinMax.class.getName());
-  /**
-   * Column group model
-   */
-  private ColumnGroupModel colGrpModel;
 
   /**
    * key generator
@@ -49,11 +44,6 @@ public class ColGroupMinMax {
   private KeyGenerator keyGenerator;
 
   /**
-   * column group id
-   */
-  private int colGroupId;
-
-  /**
    * no of column in column group
    */
   private int noOfCol;
@@ -78,12 +68,9 @@ public class ColGroupMinMax {
    */
   private byte[][] maxKeys;
 
-  public ColGroupMinMax(ColumnGroupModel colGrpModel, ColumnarSplitter columnarSplitter,
-      int colGroupId) {
-    this.colGrpModel = colGrpModel;
-    this.keyGenerator = (KeyGenerator) columnarSplitter;
-    this.colGroupId = colGroupId;
-    this.noOfCol = colGrpModel.getColumnSplit()[colGroupId];
+  public ColGroupMinMax(SegmentProperties segmentProperties, int colGroupId) {
+    this.keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGroupId);
+    this.noOfCol = segmentProperties.getNoOfColumnsInColumnGroup(colGroupId);
     min = new byte[noOfCol][];
     max = new byte[noOfCol][];
     initialise();
@@ -126,15 +113,14 @@ public class ColGroupMinMax {
       maskByteRange = new int[noOfCol][];
       maxKeys = new byte[noOfCol][];
       for (int i = 0; i < noOfCol; i++) {
-        maskByteRange[i] = getMaskByteRange(colGrpModel.getColumnGroup()[colGroupId][i]);
+        maskByteRange[i] = getMaskByteRange(i);
         // generating maxkey
-        long[] maxKey = new long[keyGenerator.getKeySizeInBytes()];
-        maxKey[colGrpModel.getColumnGroup()[colGroupId][i]] = Long.MAX_VALUE;
+        long[] maxKey = new long[noOfCol];
+        maxKey[i] = Long.MAX_VALUE;
         maxKeys[i] = keyGenerator.generateKey(maxKey);
       }
     } catch (KeyGenException e) {
-      LOGGER.error(e,
-          "Key generation failed while evaulating column group min max");
+      LOGGER.error(e, "Key generation failed while evaulating column group min max");
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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 4e49806..f1c7ad5 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
@@ -39,6 +39,7 @@ import java.util.concurrent.TimeUnit;
 
 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.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.blocklet.index.BlockletBTreeIndex;
 import org.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
@@ -171,12 +172,14 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   private int spaceReservedForBlockMetaSize;
   private FileOutputStream fileOutputStream;
 
+  private SegmentProperties segmentProperties;
+
   private List<BlockIndexInfo> blockIndexInfoList;
 
   public AbstractFactDataWriter(String storeLocation, int measureCount, int mdKeyLength,
       String databaseName, String tableName, IFileManagerComposite fileManager, int[] keyBlockSize,
       CarbonDataFileAttributes carbonDataFileAttributes, List<ColumnSchema> columnSchema,
-      String carbonDataDirectoryPath, int[] colCardinality) {
+      String carbonDataDirectoryPath, int[] colCardinality, SegmentProperties segmentProperties) {
 
     // measure count
     this.measureCount = measureCount;
@@ -187,6 +190,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     this.databaseName = databaseName;
 
     this.storeLocation = storeLocation;
+    this.segmentProperties = segmentProperties;
     this.blockletInfoList =
         new ArrayList<BlockletInfoColumnar>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
     blockIndexInfoList = new ArrayList<>();
@@ -356,7 +360,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
       CarbonFooterWriter writer = new CarbonFooterWriter(filePath);
       FileFooter convertFileMeta = CarbonMetadataUtil
           .convertFileFooter(infoList, localCardinality.length, localCardinality,
-              thriftColumnSchemaList);
+              thriftColumnSchemaList, segmentProperties);
       fillBlockIndexInfoDetails(infoList, convertFileMeta.getNum_rows(), filePath, currentPosition);
       writer.writeFooter(convertFileMeta, currentPosition);
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..24fa88d 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
@@ -24,6 +24,7 @@ import java.nio.ByteBuffer;
 import java.nio.channels.FileChannel;
 import java.util.List;
 
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.columnar.IndexStorage;
@@ -50,10 +51,11 @@ public class CarbonFactDataWriterImplForIntIndexAndAggBlock extends AbstractFact
       boolean[] aggBlocks, boolean[] isComplexType, int NoDictionaryCount,
       CarbonDataFileAttributes carbonDataFileAttributes, String databaseName,
       List<ColumnSchema> wrapperColumnSchemaList, int numberOfNoDictionaryColumn,
-      boolean[] isDictionaryColumn, String carbonDataDirectoryPath, int[] colCardinality) {
+      boolean[] isDictionaryColumn, String carbonDataDirectoryPath, int[] colCardinality,
+      SegmentProperties segmentProperties) {
     super(storeLocation, measureCount, mdKeyLength, databaseName, tableName, fileManager,
         keyBlockSize, carbonDataFileAttributes, wrapperColumnSchemaList, carbonDataDirectoryPath,
-        colCardinality);
+        colCardinality, segmentProperties);
     this.isComplexType = isComplexType;
     this.databaseName = databaseName;
     this.numberOfNoDictionaryColumn = numberOfNoDictionaryColumn;


[34/50] [abbrv] incubator-carbondata git commit: [Bug] Column Group filter row level filter and Exclude filter not working(#775)

Posted by ch...@apache.org.
[Bug] Column Group filter row level filter and Exclude filter not working(#775)



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

Branch: refs/heads/master
Commit: b5fc5188ef198402fd302b5f0a2e96c7cf077097
Parents: a743b77
Author: ashokblend <as...@gmail.com>
Authored: Wed Jun 29 23:40:45 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Wed Jun 29 23:40:45 2016 +0530

----------------------------------------------------------------------
 .../query/carbon/executor/util/QueryUtil.java   |  40 ++++
 .../executer/ColGroupFilterExecuterImpl.java    | 209 -------------------
 .../ExcludeColGroupFilterExecuterImpl.java      | 137 ++++++++++++
 .../executer/ExcludeFilterExecuterImpl.java     |  41 ++--
 .../IncludeColGroupFilterExecuterImpl.java      | 209 +++++++++++++++++++
 .../executer/RowLevelFilterExecuterImpl.java    |  84 ++++++--
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |   4 +-
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |   4 +-
 ...velRangeLessThanEqualFilterExecuterImpl.java |   4 +-
 .../RowLevelRangeLessThanFiterExecuterImpl.java |   4 +-
 .../filters/measurefilter/util/FilterUtil.java  |  32 ++-
 .../ColumnGroupDataTypesTestCase.scala          |  18 ++
 12 files changed, 518 insertions(+), 268 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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 a7d8f05..f03219f 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
@@ -63,6 +63,7 @@ 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.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 
 import org.apache.commons.lang3.ArrayUtils;
 
@@ -892,4 +893,43 @@ public class QueryUtil {
     }
     return ArrayUtils.toPrimitive(indexList.toArray(new Integer[indexList.size()]));
   }
+
+  /**
+   * It is required for extracting column data from columngroup chunk
+   *
+   * @return
+   * @throws KeyGenException
+   */
+  public static KeyStructureInfo getKeyStructureInfo(SegmentProperties segmentProperties,
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo) throws KeyGenException {
+    int colGrpId = getColumnGroupId(segmentProperties, dimColumnEvaluatorInfo.getColumnIndex());
+    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+
+    mdKeyOrdinal.add(segmentProperties
+        .getColumnGroupMdKeyOrdinal(colGrpId, dimColumnEvaluatorInfo.getColumnIndex()));
+    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
+    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
+    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
+    KeyStructureInfo restructureInfos = new KeyStructureInfo();
+    restructureInfos.setKeyGenerator(keyGenerator);
+    restructureInfos.setMaskByteRanges(maskByteRanges);
+    restructureInfos.setMaxKey(maxKey);
+    restructureInfos.setMaskedBytes(maksedByte);
+    return restructureInfos;
+  }
+
+  public static int getColumnGroupId(SegmentProperties segmentProperties, int ordinal) {
+    int[][] columnGroups = segmentProperties.getColumnGroups();
+    int colGrpId = -1;
+    for (int i = 0; i < columnGroups.length; i++) {
+      if (columnGroups[i].length > 1) {
+        colGrpId++;
+        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
+          break;
+        }
+      }
+    }
+    return colGrpId;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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 98b3ed4..0000000
--- a/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
+++ /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.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.keygenerator.KeyGenerator;
-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 {
-    int colGrpId = getColumnGroupId(dimColumnEvaluatorInfo.getColumnIndex());
-    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
-    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
-    mdKeyOrdinal.add(getMdkeyOrdinal(dimColumnEvaluatorInfo.getColumnIndex(), colGrpId));
-    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
-    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
-    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
-    KeyStructureInfo restructureInfos = new KeyStructureInfo();
-    restructureInfos.setKeyGenerator(keyGenerator);
-    restructureInfos.setMaskByteRanges(maskByteRanges);
-    restructureInfos.setMaxKey(maxKey);
-    restructureInfos.setMaskedBytes(maksedByte);
-    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 colGrpColumns
-   * @param minMaxData
-   * @param columnIndex
-   * @return
-   */
-  private byte[] getMinMaxData(int[] colGrpColumns, byte[] minMaxData, int columnIndex) {
-    int startIndex = 0;
-    int endIndex = 0;
-    if (null != colGrpColumns) {
-      for (int i = 0; i < colGrpColumns.length; i++) {
-        int colGrpId = getColumnGroupId(colGrpColumns[i]);
-        int mdKeyOrdinal = getMdkeyOrdinal(colGrpColumns[i], colGrpId);
-        int[] byteRange = getKeyGenerator(colGrpId).getKeyByteOffsets(mdKeyOrdinal);
-        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;
-  }
-
-  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
-    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
-  }
-
-  private int getColumnGroupId(int ordinal) {
-    int[][] columnGroups = segmentProperties.getColumnGroups();
-    int colGrpId = -1;
-    for (int i = 0; i < columnGroups.length; i++) {
-      if (columnGroups[i].length > 1) {
-        colGrpId++;
-        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
-          break;
-        }
-      }
-    }
-    return colGrpId;
-  }
-
-  public KeyGenerator getKeyGenerator(int colGrpId) {
-    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
-  }
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/core/src/main/java/org/carbondata/query/filter/executer/ExcludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/ExcludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/ExcludeColGroupFilterExecuterImpl.java
new file mode 100644
index 0000000..0a73ca9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/filter/executer/ExcludeColGroupFilterExecuterImpl.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.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.keygenerator.KeyGenerator;
+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 ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(ExcludeColGroupFilterExecuterImpl.class.getName());
+
+  /**
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   */
+  public ExcludeColGroupFilterExecuterImpl(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);
+    bitSet.flip(0, 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.flip(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 {
+    int colGrpId = getColumnGroupId(dimColEvaluatorInfo.getColumnIndex());
+    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+    mdKeyOrdinal.add(getMdkeyOrdinal(dimColEvaluatorInfo.getColumnIndex(), colGrpId));
+    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
+    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
+    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
+    KeyStructureInfo restructureInfos = new KeyStructureInfo();
+    restructureInfos.setKeyGenerator(keyGenerator);
+    restructureInfos.setMaskByteRanges(maskByteRanges);
+    restructureInfos.setMaxKey(maxKey);
+    restructureInfos.setMaskedBytes(maksedByte);
+    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);
+    bitSet.flip(0, 1);
+    return bitSet;
+  }
+
+  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
+    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
+  }
+
+  private int getColumnGroupId(int ordinal) {
+    int[][] columnGroups = segmentProperties.getColumnGroups();
+    int colGrpId = -1;
+    for (int i = 0; i < columnGroups.length; i++) {
+      if (columnGroups[i].length > 1) {
+        colGrpId++;
+        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
+          break;
+        }
+      }
+    }
+    return colGrpId;
+  }
+
+  public KeyGenerator getKeyGenerator(int colGrpId) {
+    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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
index f2c6de3..1f620c8 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
@@ -21,10 +21,10 @@ 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.keygenerator.KeyGenerator;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.carbon.processor.BlocksChunkHolder;
@@ -34,37 +34,38 @@ import org.carbondata.query.filters.measurefilter.util.FilterUtil;
 
 public class ExcludeFilterExecuterImpl implements FilterExecuter {
 
-  DimColumnResolvedFilterInfo dimColEvaluatorInfo;
-  DimColumnExecuterFilterInfo dimColumnExecuterInfo;
-
-  public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo) {
-    this.dimColEvaluatorInfo = dimColEvaluatorInfo;
-  }
+  protected DimColumnResolvedFilterInfo dimColEvaluatorInfo;
+  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
+  protected SegmentProperties segmentProperties;
 
   public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
-      KeyGenerator blockKeyGenerator) {
-    this(dimColEvaluatorInfo);
+      SegmentProperties segmentProperties) {
+    this.dimColEvaluatorInfo = dimColEvaluatorInfo;
     dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
-    FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(), blockKeyGenerator,
-        dimColEvaluatorInfo.getDimension(), dimColumnExecuterInfo);
+    this.segmentProperties = segmentProperties;
+    FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(),
+        segmentProperties.getDimensionKeyGenerator(), dimColEvaluatorInfo.getDimension(),
+        dimColumnExecuterInfo);
   }
 
   @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) {
-    if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
-      blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
-          dimColEvaluatorInfo.getColumnIndex());
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+        .get(dimColEvaluatorInfo.getColumnIndex());
+    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
+      blockChunkHolder.getDataBlock()
+          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
     }
-    if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
-      blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()] =
-          blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
-              dimColEvaluatorInfo.getColumnIndex());
+    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
+      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
+          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
     }
     return getFilteredIndexes(
-        blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()],
+        blockChunkHolder.getDimensionDataChunk()[blockIndex],
         blockChunkHolder.getDataBlock().nodeSize());
   }
 
-  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimColumnDataChunk, int numerOfRows) {
+  protected BitSet getFilteredIndexes(DimensionColumnDataChunk dimColumnDataChunk,
+      int numerOfRows) {
     // For high cardinality dimensions.
     if (dimColumnDataChunk.getAttributes().isNoDictionary()
         && dimColumnDataChunk instanceof VariableLengthDimensionDataChunk) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/core/src/main/java/org/carbondata/query/filter/executer/IncludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/query/filter/executer/IncludeColGroupFilterExecuterImpl.java b/core/src/main/java/org/carbondata/query/filter/executer/IncludeColGroupFilterExecuterImpl.java
new file mode 100644
index 0000000..f230484
--- /dev/null
+++ b/core/src/main/java/org/carbondata/query/filter/executer/IncludeColGroupFilterExecuterImpl.java
@@ -0,0 +1,209 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF 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.keygenerator.KeyGenerator;
+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 IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(IncludeColGroupFilterExecuterImpl.class.getName());
+
+  /**
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   */
+  public IncludeColGroupFilterExecuterImpl(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 {
+    int colGrpId = getColumnGroupId(dimColumnEvaluatorInfo.getColumnIndex());
+    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
+    mdKeyOrdinal.add(getMdkeyOrdinal(dimColumnEvaluatorInfo.getColumnIndex(), colGrpId));
+    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
+    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
+    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
+    KeyStructureInfo restructureInfos = new KeyStructureInfo();
+    restructureInfos.setKeyGenerator(keyGenerator);
+    restructureInfos.setMaskByteRanges(maskByteRanges);
+    restructureInfos.setMaxKey(maxKey);
+    restructureInfos.setMaskedBytes(maksedByte);
+    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 colGrpColumns
+   * @param minMaxData
+   * @param columnIndex
+   * @return
+   */
+  private byte[] getMinMaxData(int[] colGrpColumns, byte[] minMaxData, int columnIndex) {
+    int startIndex = 0;
+    int endIndex = 0;
+    if (null != colGrpColumns) {
+      for (int i = 0; i < colGrpColumns.length; i++) {
+        int colGrpId = getColumnGroupId(colGrpColumns[i]);
+        int mdKeyOrdinal = getMdkeyOrdinal(colGrpColumns[i], colGrpId);
+        int[] byteRange = getKeyGenerator(colGrpId).getKeyByteOffsets(mdKeyOrdinal);
+        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;
+  }
+
+  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
+    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
+  }
+
+  private int getColumnGroupId(int ordinal) {
+    int[][] columnGroups = segmentProperties.getColumnGroups();
+    int colGrpId = -1;
+    for (int i = 0; i < columnGroups.length; i++) {
+      if (columnGroups[i].length > 1) {
+        colGrpId++;
+        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
+          break;
+        }
+      }
+    }
+    return colGrpId;
+  }
+
+  public KeyGenerator getKeyGenerator(int colGrpId) {
+    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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 095f757..f08936b 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
@@ -28,16 +28,20 @@ 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.block.SegmentProperties;
 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.KeyGenException;
 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.executor.infos.KeyStructureInfo;
+import org.carbondata.query.carbon.executor.util.QueryUtil;
 import org.carbondata.query.carbon.processor.BlocksChunkHolder;
 import org.carbondata.query.carbon.util.DataTypeUtil;
 import org.carbondata.query.carbonfilterinterface.RowImpl;
@@ -58,11 +62,22 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
   protected List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList;
   protected Expression exp;
   protected AbsoluteTableIdentifier tableIdentifier;
+  protected SegmentProperties segmentProperties;
+  /**
+   * it has index at which given dimension is stored in file
+   */
+  private int[] blocksIndex;
 
   public RowLevelFilterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier) {
+      AbsoluteTableIdentifier tableIdentifier, SegmentProperties segmentProperties) {
     this.dimColEvaluatorInfoList = dimColEvaluatorInfoList;
+    this.segmentProperties = segmentProperties;
+    this.blocksIndex = new int[dimColEvaluatorInfoList.size()];
+    for (int i=0;i<dimColEvaluatorInfoList.size();i++) {
+      this.blocksIndex[i] = segmentProperties.getDimensionOrdinalToBlockMapping()
+              .get(dimColEvaluatorInfoList.get(i).getColumnIndex());
+    }
     if (null == msrColEvalutorInfoList) {
       this.msrColEvalutorInfoList = new ArrayList<MeasureColumnResolvedFilterInfo>(20);
     } else {
@@ -74,18 +89,17 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
 
   @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
       throws FilterUnsupportedException {
-    for (DimColumnResolvedFilterInfo dimColumnEvaluatorInfo : dimColEvaluatorInfoList) {
+    for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
       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());
+        if (null == blockChunkHolder.getDimensionDataChunk()[blocksIndex[i]]) {
+          blockChunkHolder.getDimensionDataChunk()[blocksIndex[i]] = blockChunkHolder.getDataBlock()
+              .getDimensionChunk(blockChunkHolder.getFileReader(), blocksIndex[i]);
         }
       } else {
         GenericQueryType complexType = dimColumnEvaluatorInfo.getComplexTypesWithBlockStartIndex()
-            .get(dimColumnEvaluatorInfo.getColumnIndex());
+            .get(blocksIndex[i]);
         complexType.fillRequiredBlockData(blockChunkHolder);
       }
     }
@@ -143,19 +157,20 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
       throws QueryExecutionException {
     Object[] record = new Object[dimColEvaluatorInfoList.size() + msrColEvalutorInfoList.size()];
     String memberString = null;
-    for (DimColumnResolvedFilterInfo dimColumnEvaluatorInfo : dimColEvaluatorInfoList) {
+    for (int i=0;i<dimColEvaluatorInfoList.size();i++) {
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
       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) {
+            && blockChunkHolder
+            .getDimensionDataChunk()[blocksIndex[i]] instanceof VariableLengthDimensionDataChunk) {
 
           VariableLengthDimensionDataChunk dimensionColumnDataChunk =
               (VariableLengthDimensionDataChunk) blockChunkHolder
-                  .getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()];
+                  .getDimensionDataChunk()[blocksIndex[i]];
           if (null != dimensionColumnDataChunk.getCompleteDataChunk()) {
             memberString =
                 readMemberBasedOnNoDictionaryVal(dimColumnEvaluatorInfo, dimensionColumnDataChunk,
@@ -173,7 +188,8 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
           }
         } else {
           int dictionaryValue =
-              readSurrogatesFromColumnBlock(blockChunkHolder, index, dimColumnEvaluatorInfo);
+              readSurrogatesFromColumnBlock(blockChunkHolder, index, dimColumnEvaluatorInfo,
+                  blocksIndex[i]);
           Dictionary forwardDictionary = null;
           if (dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DICTIONARY)
               && !dimColumnEvaluatorInfo.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
@@ -317,13 +333,41 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
    * @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;
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int blockIndex) {
+    if (dimColumnEvaluatorInfo.getDimension().isColumnar()) {
+      byte[] rawData = blockChunkHolder.getDimensionDataChunk()[blockIndex].getChunkData(index);
+      ByteBuffer byteBuffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
+      int dictionaryValue = CarbonUtil.getSurrogateKey(rawData, byteBuffer);
+      return dictionaryValue;
+    } else {
+      return readSurrogatesFromColumnGroupBlock(blockChunkHolder, index, dimColumnEvaluatorInfo,
+          blockIndex);
+    }
+
+  }
+
+  /**
+   * @param blockChunkHolder
+   * @param index
+   * @param dimColumnEvaluatorInfo
+   * @return read surrogate of given row of given column group dimension
+   */
+  private int readSurrogatesFromColumnGroupBlock(BlocksChunkHolder blockChunkHolder, int index,
+      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int blockIndex) {
+    try {
+      KeyStructureInfo keyStructureInfo =
+          QueryUtil.getKeyStructureInfo(segmentProperties, dimColumnEvaluatorInfo);
+      byte[] colData = blockChunkHolder.getDimensionDataChunk()[blockIndex].getChunkData(index);
+      long[] result = keyStructureInfo.getKeyGenerator().getKeyArray(colData);
+      int colGroupId =
+          QueryUtil.getColumnGroupId(segmentProperties, dimColumnEvaluatorInfo.getColumnIndex());
+      int dictionaryValue = (int) result[segmentProperties
+          .getColumnGroupMdKeyOrdinal(colGroupId, dimColumnEvaluatorInfo.getColumnIndex())];
+      return dictionaryValue;
+    } catch (KeyGenException e) {
+      LOGGER.error(e);
+    }
+    return 0;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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
index 571f046..0efca00 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -36,16 +36,14 @@ import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFi
 
 public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
   private byte[][] filterRangeValues;
-  private SegmentProperties segmentProperties;
 
   public RowLevelRangeGrtThanFiterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
       SegmentProperties segmentProperties) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
     this.filterRangeValues = filterRangeValues;
-    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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
index e390b8d..935c2ce 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -37,16 +37,14 @@ import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFi
 public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl {
 
   protected byte[][] filterRangeValues;
-  private SegmentProperties segmentProperties;
 
   public RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
       SegmentProperties segmentProperties) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
     this.filterRangeValues = filterRangeValues;
-    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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
index 1f33c8c..27f7935 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -36,16 +36,14 @@ import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFi
 
 public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilterExecuterImpl {
   protected byte[][] filterRangeValues;
-  private SegmentProperties segmentProperties;
 
   public RowLevelRangeLessThanEqualFilterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
       SegmentProperties segmentProperties) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
     this.filterRangeValues = filterRangeValues;
-    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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
index 9bb0420..1786553 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -36,16 +36,14 @@ import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFi
 
 public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
   private byte[][] filterRangeValues;
-  private SegmentProperties segmentProperties;
 
   public RowLevelRangeLessThanFiterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
       SegmentProperties segmentProperties) {
-    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
+    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
     this.filterRangeValues = filterRangeValues;
-    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/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
index ee72f62..8bae3fc 100644
--- 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
@@ -73,9 +73,10 @@ import org.carbondata.query.expression.LiteralExpression;
 import org.carbondata.query.expression.exception.FilterIllegalMemberException;
 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.ExcludeColGroupFilterExecuterImpl;
 import org.carbondata.query.filter.executer.ExcludeFilterExecuterImpl;
 import org.carbondata.query.filter.executer.FilterExecuter;
+import org.carbondata.query.filter.executer.IncludeColGroupFilterExecuterImpl;
 import org.carbondata.query.filter.executer.IncludeFilterExecuterImpl;
 import org.carbondata.query.filter.executer.OrFilterExecuterImpl;
 import org.carbondata.query.filter.executer.RestructureFilterExecuterImpl;
@@ -113,9 +114,8 @@ public final class FilterUtil {
           return getIncludeFilterExecuter(
               filterExpressionResolverTree.getDimColResolvedFilterInfo(), segmentProperties);
         case EXCLUDE:
-          return new ExcludeFilterExecuterImpl(
-              filterExpressionResolverTree.getDimColResolvedFilterInfo(),
-              segmentProperties.getDimensionKeyGenerator());
+          return getExcludeFilterExecuter(
+              filterExpressionResolverTree.getDimColResolvedFilterInfo(), segmentProperties);
         case OR:
           return new OrFilterExecuterImpl(
               createFilterExecuterTree(filterExpressionResolverTree.getLeft(), segmentProperties),
@@ -143,7 +143,8 @@ public final class FilterUtil {
               ((RowLevelFilterResolverImpl) filterExpressionResolverTree)
                   .getMsrColEvalutorInfoList(),
               ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getFilterExpresion(),
-              ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier());
+              ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+              segmentProperties);
 
       }
     }
@@ -151,7 +152,8 @@ public final class FilterUtil {
         ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getDimColEvaluatorInfoList(),
         ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getMsrColEvalutorInfoList(),
         ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getFilterExpresion(),
-        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier());
+        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
+        segmentProperties);
 
   }
 
@@ -168,11 +170,27 @@ public final class FilterUtil {
     if (dimColResolvedFilterInfo.getDimension().isColumnar()) {
       return new IncludeFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
     } else {
-      return new ColGroupFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
+      return new IncludeColGroupFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
     }
   }
 
   /**
+   * It gives filter executer based on columnar or column group
+   *
+   * @param dimColResolvedFilterInfo
+   * @param segmentProperties
+   * @return
+   */
+  private static FilterExecuter getExcludeFilterExecuter(
+      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties) {
+
+    if (dimColResolvedFilterInfo.getDimension().isColumnar()) {
+      return new ExcludeFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
+    } else {
+      return new ExcludeColGroupFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
+    }
+  }
+  /**
    * This method will check if a given expression contains a column expression
    * recursively.
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/b5fc5188/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
index 57f296a..8098308 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
@@ -104,6 +104,24 @@ class ColumnGroupDataTypesTestCase extends QueryTest with BeforeAndAfterAll {
       sql("select * from colgrp_dictexclude_after where column3='column311' and column4='column42' "),
       sql("select * from normal where column3='column311' and column4='column42'"))
   }
+  test("ExcludeFilter") {
+    checkAnswer(
+      sql("select * from colgrp where column3 != 'column311'"),
+      sql("select * from normal where column3 != 'column311'"))
+
+    checkAnswer(
+      sql("select * from colgrp where column3 like 'column31%'"),
+      sql("select * from normal where column3 like 'column31%'"))
+    checkAnswer(
+      sql("select * from colgrp where column3 not like 'column31%'"),
+      sql("select * from normal where column3 not like 'column31%'"))
+  }
+  test("RowFilter") {
+    checkAnswer(
+      sql("select * from colgrp where column3 != column4"),
+      sql("select * from normal where column3 != column4"))
+  }
+
   override def afterAll {
     sql("drop table colgrp")
     sql("drop table normal")


[03/50] [abbrv] incubator-carbondata git commit: Merge pull request #744 from HuaweiBigData/update_readme

Posted by ch...@apache.org.
Merge pull request #744 from HuaweiBigData/update_readme

Update README.md

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

Branch: refs/heads/master
Commit: 6eb5728b01d2bbc65f02b28d6743869e60095235
Parents: b0f2f06 acbacb8
Author: Liang Chen <ch...@huawei.com>
Authored: Sat Jun 25 03:01:44 2016 +0530
Committer: GitHub <no...@github.com>
Committed: Sat Jun 25 03:01:44 2016 +0530

----------------------------------------------------------------------
 README.md | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------



[17/50] [abbrv] incubator-carbondata git commit: [Bug] Timestamp with different format compare to spark date format will fail to provide filter result. this is because while (#754)

Posted by ch...@apache.org.
[Bug] Timestamp with different format compare to spark date format will fail to provide filter result. this is because while (#754)

querying the engine is trying to get the direct surrogate based in user defined time format in properties file.

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

Branch: refs/heads/master
Commit: 5b6081d7b203411fcc2e4b941c6f67f62d8c2878
Parents: 73975b0
Author: sujith71955 <su...@gmail.com>
Authored: Sun Jun 26 22:09:17 2016 +0800
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sun Jun 26 19:39:17 2016 +0530

----------------------------------------------------------------------
 .../DirectDictionaryGenerator.java              | 13 ++++++++
 .../TimeStampDirectDictionaryGenerator.java     | 32 ++++++++++++++++----
 .../query/expression/ExpressionResult.java      |  9 ++++--
 .../visitor/CustomTypeDictionaryVisitor.java    |  4 ++-
 .../src/test/resources/data2_DiffTimeFormat.csv |  4 +++
 .../filterexpr/FilterProcessorTestCase.scala    | 25 +++++++++++++++
 6 files changed, 77 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5b6081d7/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
index 476d09f..145c89a 100644
--- a/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
+++ b/core/src/main/java/org/carbondata/core/keygenerator/directdictionary/DirectDictionaryGenerator.java
@@ -40,4 +40,17 @@ public interface DirectDictionaryGenerator {
    */
   Object getValueFromSurrogate(int key);
 
+  /**
+   * The method generate and returns the dictionary / surrogate key for direct dictionary column
+   * This Method is called while executing filter queries for getting direct surrogate members.
+   * Currently the query engine layer only supports yyyy-MM-dd HH:mm:ss date format no matter
+   * in which format the data is been stored, so while retrieving the direct surrogate value for
+   * filter member first it should be converted in date form as per above format and needs to
+   * retrieve time stamp.
+   *
+   * @param member The member string value
+   * @return returns dictionary/ surrogate value
+   */
+  int generateDirectSurrogateKey(String memberStr, String format);
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5b6081d7/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 8493991..97baf3d 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,12 +40,6 @@ import static org.carbondata.core.keygenerator.directdictionary.timestamp.TimeSt
 public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
 
   /**
-   * Logger instance
-   */
-  private static final LogService LOGGER =
-      LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
-
-  /**
    * The value of 1 unit of the SECOND, MINUTE, HOUR, or DAY in millis.
    */
   public static final long granularityFactor;
@@ -55,6 +49,11 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
    * customized the start of position. for example "January 1, 2000"
    */
   public static final long cutOffTimeStamp;
+  /**
+   * Logger instance
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
 
   /**
    * initialization block for granularityFactor and cutOffTimeStamp
@@ -119,6 +118,27 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
         .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
       return 1;
     }
+    return getDirectSurrogateForMember(memberStr, timeParser);
+  }
+
+  /**
+   * The method take member String as input and converts
+   * and returns the dictionary key
+   *
+   * @param memberStr date format string
+   * @return dictionary value
+   */
+  public int generateDirectSurrogateKey(String memberStr, String format) {
+    SimpleDateFormat timeParser = new SimpleDateFormat(format);
+    timeParser.setLenient(false);
+    if (null == memberStr || memberStr.trim().isEmpty() || memberStr
+        .equals(CarbonCommonConstants.MEMBER_DEFAULT_VAL)) {
+      return 1;
+    }
+    return getDirectSurrogateForMember(memberStr, timeParser);
+  }
+
+  private int getDirectSurrogateForMember(String memberStr, SimpleDateFormat timeParser) {
     Date dateToStr = null;
     try {
       dateToStr = timeParser.parse(memberStr);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5b6081d7/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
index 1f258e2..067f2d3 100644
--- a/core/src/main/java/org/carbondata/query/expression/ExpressionResult.java
+++ b/core/src/main/java/org/carbondata/query/expression/ExpressionResult.java
@@ -243,9 +243,12 @@ public class ExpressionResult implements Comparable<ExpressionResult> {
     try {
       switch (this.getDataType()) {
         case StringType:
-          SimpleDateFormat parser = new SimpleDateFormat(CarbonProperties.getInstance()
-              .getProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT,
-                  CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+          // Currently the query engine layer only supports yyyy-MM-dd HH:mm:ss date format
+          // no matter in which format the data is been stored, so while retrieving the direct
+          // surrogate value for filter member first it should be converted in date form as per
+          // above format and needs to retrieve time stamp.
+          SimpleDateFormat parser =
+              new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
           Date dateToStr;
           try {
             dateToStr = parser.parse(value.toString());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5b6081d7/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
index 0aec1c1..81808e8 100644
--- 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
@@ -25,6 +25,7 @@ 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.constants.CarbonCommonConstants;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
 import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.carbondata.query.expression.ColumnExpression;
@@ -71,7 +72,8 @@ public class CustomTypeDictionaryVisitor implements ResolvedFilterInfoVisitorInt
         .getDirectDictionaryGenerator(columnExpression.getDimension().getDataType());
     // Reading the dictionary value direct
     for (String filterMember : evaluateResultListFinal) {
-      surrogates.add(directDictionaryGenerator.generateDirectSurrogateKey(filterMember));
+      surrogates.add(directDictionaryGenerator.generateDirectSurrogateKey(filterMember,
+          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
     }
     Collections.sort(surrogates);
     DimColumnFilterInfo columnFilterInfo = null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5b6081d7/integration/spark/src/test/resources/data2_DiffTimeFormat.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/data2_DiffTimeFormat.csv b/integration/spark/src/test/resources/data2_DiffTimeFormat.csv
new file mode 100644
index 0000000..c5fe230
--- /dev/null
+++ b/integration/spark/src/test/resources/data2_DiffTimeFormat.csv
@@ -0,0 +1,4 @@
+ID,date,country,name,phonetype,serialname,salary
+4,07-10-2014 00:00:00,china,aaa4,phone2435,ASD66902,4
+8,07-20-2014 00:00:00,china,aaa5,phone2441,ASD90633,10
+6,07-25-2014 00:00:00,china,aaa6,phone294,ASD59961,15005
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5b6081d7/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
index 325800b..41930df 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
@@ -40,10 +40,34 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists filtertestTables")
     sql("drop table if exists filtertestTablesWithDecimal")
     sql("drop table if exists filtertestTablesWithNull")
+    sql("drop table if exists filterWithTimeStamp")
     sql("CREATE TABLE filtertestTables (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, "MM-dd-yyyy HH:mm:ss")
+        
+    sql("CREATE TABLE filterWithTimeStamp (ID int, date Timestamp, country String, " +
+      "name String, phonetype String, serialname String, salary int) " +
+        "STORED BY 'org.apache.carbondata.format'"
+    )
+    sql(
+      s"LOAD DATA LOCAL INPATH './src/test/resources/data2_DiffTimeFormat.csv' INTO TABLE " +
+        s"filterWithTimeStamp " +
+        s"OPTIONS('DELIMITER'= ',', " +
+        s"'FILEHEADER'= '')"
+    )
+    
+     test("Time stamp filter with diff time format for load ") {
+    checkAnswer(
+      sql("select date  from filterWithTimeStamp where date > '2014-07-10 00:00:00'"),
+      Seq(Row(Timestamp.valueOf("2014-07-20 00:00:00.0")),
+        Row(Timestamp.valueOf("2014-07-25 00:00:00.0"))
+      )
+    )
+  }
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
     sql(
@@ -79,6 +103,7 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     )
   }
 
+    
   test("Is not null filter") {
     checkAnswer(
       sql("select id from filtertestTablesWithNull " + "where id is not null"),


[27/50] [abbrv] incubator-carbondata git commit: [Bug] Removed all cube related DDL/DMLs (#759)

Posted by ch...@apache.org.
[Bug] Removed all cube related DDL/DMLs (#759)

* Removed all cube related DDL/DMLs
* Corrected test cases

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

Branch: refs/heads/master
Commit: 9d846e415d5fe7d5f0462d5b28e12a6397c118d7
Parents: e689092
Author: Manu <ma...@gmail.com>
Authored: Mon Jun 27 21:58:05 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Mon Jun 27 21:58:05 2016 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/CarbonSqlParser.scala  | 243 +------------------
 .../DataCompactionCardinalityBoundryTest.scala  |  12 +-
 .../datacompaction/DataCompactionLockTest.scala |  18 +-
 .../datacompaction/DataCompactionTest.scala     |  12 +-
 .../HighCardinalityDataTypesTestCase.scala      |  55 +++--
 .../filterexpr/FilterProcessorTestCase.scala    |   2 +-
 .../NullMeasureValueTestCaseFilter.scala        |   2 +-
 .../NullMeasureValueTestCaseAggregate.scala     |   2 +-
 8 files changed, 59 insertions(+), 287 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/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 e766aaa..69026ee 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
@@ -179,23 +179,10 @@ class CarbonSqlParser()
   }
 
   override protected lazy val start: Parser[LogicalPlan] =
-     createCube | showCreateCube | loadManagement | createAggregateTable |
-      describeTable |
-      showCube | showLoads | alterCube | showAllCubes | alterTable | createTable
-
-  protected lazy val loadManagement: Parser[LogicalPlan] = loadData | dropCubeOrTable |
-    deleteLoadsByID | deleteLoadsByLoadDate | deleteLoadsByDate | cleanFiles | loadDataNew
-
-  protected lazy val createAggregateTable: Parser[LogicalPlan] =
-    CREATE ~> AGGREGATETABLE ~>
-      (aggregates) ~
-      (FROM ~> CUBE ~> (ident <~ ".").? ~ ident) <~ opt(";") ^^ {
-      case aggregates ~ cube =>
-        cube match {
-          case schemaName ~ cubeName =>
-            AddAggregatesToTable(schemaName, cubeName.toLowerCase(), aggregates)
-        }
-    }
+    loadManagement | describeTable | showLoads | alterTable | createTable
+
+  protected lazy val loadManagement: Parser[LogicalPlan] = deleteLoadsByID | deleteLoadsByLoadDate |
+    cleanFiles | loadDataNew
 
   protected lazy val aggregates: Parser[Seq[AggregateTableAttributes]] =
     repsep((aggregateExpression | aggAttribute), ",")
@@ -248,21 +235,6 @@ class CarbonSqlParser()
   protected lazy val dropDefinition =
     DROP ~> "(" ~> rep1sep((stringLit | ident), ",") <~ ")"
 
-  protected lazy val addDefinition =
-    (ADD ~> cubeDefinition)
-
-  protected lazy val cubeDefinition =
-    ((DIMENSIONS ~> "(" ~> dimCols <~ ")").? ~
-      (MEASURES ~> "(" ~> measureCols <~ ")").? ~
-      (WITH ~ (simpleDimRelations)).?)
-
-  protected lazy val showCubeDefinition =
-    ((DIMENSIONS ~> "(" ~> dimCols <~ ")").? ~
-      (MEASURES ~> "(" ~> measureCols <~ ")").? ~
-      ((FACT ~> FROM ~ (dbTableIdentifier | stringLit) ~ (colsFilter).? ~
-        ("," ~> DIMENSION ~> FROM ~> dimRelations).?).?) ~
-      (WITH ~ (simpleDimRelations)).?)
-
   protected lazy val aggOptionsForShowCreate =
     (aggregation).? ~ (",".? ~> partitioner).?
   protected lazy val aggOptions =
@@ -275,48 +247,6 @@ class CarbonSqlParser()
 
   protected val escapedIdentifier = "`([^`]+)`".r
 
-
-  protected lazy val showCreateCube: Parser[LogicalPlan] =
-    SHOW ~> CREATE ~> CUBE ~> (IF ~> NOT ~> EXISTS).? ~ (ident <~ ".").? ~ ident ~
-      showCubeDefinition ~
-      (OPTIONS ~> showcreateCubeOptionDef).? <~ (";").? ^^ {
-      case exists ~ schemaName ~ cubeName ~ cubeDefinition ~ options =>
-        val (dimCols, msrCols, fromKeyword, withKeyword, source,
-        factFieldsList, dimRelations, simpleDimRelations) =
-          cubeDefinition match {
-            case _ ~ _ ~ Some(_) ~ Some(_) => sys
-              .error("FROM and WITH keywords can not be used together")
-
-            case dimCols ~ msrCols ~ fromBody ~ withBody =>
-              val (fromKeyword, source, factFieldsList, dimRelations) = fromBody match {
-                case Some(fromKeyword ~ source ~ factFieldsList ~ dimRelations) =>
-                  (fromKeyword, source, factFieldsList, dimRelations)
-
-                case _ => ("", "", None, None)
-              }
-
-              val (withKeyword, simpleDimRelations) = withBody match {
-                case Some(withKeyword ~ simpleDimRelations) => (withKeyword, simpleDimRelations)
-                case _ => ("", Seq())
-              }
-
-              (dimCols.getOrElse(Seq()), msrCols
-                .getOrElse(Seq()), fromKeyword, withKeyword, source, factFieldsList,
-                dimRelations.getOrElse(Seq()), simpleDimRelations)
-          }
-
-        val (aggregation, partitioner) = options match {
-          case Some(aggregation ~ partitioner) => (aggregation.getOrElse(Seq()), partitioner)
-          case _ => (Seq(), None)
-        }
-
-        ShowCreateCubeCommand(tableModel(exists.isDefined,
-          schemaName.getOrElse("default"), schemaName, cubeName.toLowerCase(),
-          reorderDimensions(dimCols.map(f => normalizeType(f)).map(f => addParent(f))),
-          msrCols.map(f => normalizeType(f)), fromKeyword, withKeyword, source,
-          factFieldsList, dimRelations, simpleDimRelations, None, aggregation, partitioner, null))
-    }
-
   private def reorderDimensions(dims: Seq[Field]): Seq[Field] = {
     var complexDimensions: Seq[Field] = Seq()
     var dimensions: Seq[Field] = Seq()
@@ -954,99 +884,6 @@ class CarbonSqlParser()
     case other => other
   }
 
-  protected lazy val createCube: Parser[LogicalPlan] =
-    CREATE ~> CUBE ~> (IF ~> NOT ~> EXISTS).? ~ (ident <~ ".").? ~ ident ~
-      cubeDefinition ~
-      (OPTIONS ~> createCubeOptionDef).? <~ (";").? ^^ {
-      case exists ~ schemaName ~ cubeName ~ cubeDefinition ~ options =>
-        val (dimCols, msrCols, withKeyword, simpleDimRelations) = cubeDefinition match {
-
-          case dimCols ~ msrCols ~ withBody =>
-            val (withKeyword, simpleDimRelations) = withBody match {
-              case Some(withKeyword ~ simpleDimRelations) => (withKeyword, simpleDimRelations)
-              case _ => ("", Seq())
-            }
-
-            (dimCols.getOrElse(Seq()), msrCols.getOrElse(Seq()), withKeyword, simpleDimRelations)
-        }
-
-        val (highCard, aggregation, partitioner) = options match {
-          case Some(hc ~ agg ~ part) => (hc.getOrElse(Some(Seq())), agg.getOrElse(Seq()), part)
-          case _ => (Some(Seq()), Seq(), None)
-        }
-
-        CreateCube(tableModel(exists.isDefined,
-          schemaName.getOrElse("default"), schemaName, cubeName.toLowerCase(),
-          reorderDimensions(dimCols.map(f => normalizeType(f)).map(f => addParent(f))),
-          msrCols.map(f => normalizeType(f)), "", withKeyword, "",
-          None, Seq(), simpleDimRelations, highCard, aggregation, partitioner, null))
-    }
-
-  protected lazy val alterCube: Parser[LogicalPlan] =
-    ALTER ~> CUBE ~> (ident <~ ".").? ~ ident ~
-      (dropDefinition).? ~
-      (addDefinition).? ~
-      (OPTIONS ~> defaultOptions).? <~ opt(";") ^^ {
-      case schemaName ~ cubeName ~ dropDefinition ~ addDefinition ~ options =>
-        val (dimCols, msrCols, withKeyword, simpleDimRelations) = addDefinition match {
-
-          case Some(dimCols ~ msrCols ~ withBody) =>
-            val (withKeyword, simpleDimRelations) = withBody match {
-              case Some(withKeyword ~ simpleDimRelations) => (withKeyword, simpleDimRelations)
-              case _ => ("", Seq())
-            }
-
-            if (dimCols.isEmpty && msrCols.isEmpty) {
-              sys.error(
-                "empty ADD definition found.Please provide the dimensions/measures to be added.")
-            } else {
-              (dimCols.getOrElse(Seq()), msrCols.getOrElse(Seq()), withKeyword, simpleDimRelations)
-            }
-
-          case _ =>
-            (Seq(), Seq(), "", Seq())
-        }
-
-        val (noDictionary, aggregation, defaultVals) = options match {
-          case Some(noDictionary ~ aggregation ~ defaultVals) => (noDictionary
-            .getOrElse(Some(Seq())),
-            aggregation.getOrElse(Seq()), defaultVals.getOrElse(Seq()))
-          case _ => (Some(Seq()), Seq(), Seq())
-        }
-
-        val (dropCols) = dropDefinition match {
-          case Some(dropCols) => (dropCols)
-          case _ => (Seq())
-        }
-
-        AlterTable(tableModel(false,
-          schemaName.getOrElse("default"),
-          schemaName, cubeName.toLowerCase(),
-          dimCols.map(f => normalizeType(f)),
-          msrCols.map(f => normalizeType(f)), "", withKeyword, "",
-          None, Seq(), simpleDimRelations, noDictionary, aggregation, None, null),
-          dropCols, defaultVals)
-
-      case _ =>
-        sys.error("Parsing error")
-    }
-
-
-  protected lazy val loadData: Parser[LogicalPlan] =
-    LOAD ~> DATA ~> FACT ~> FROM ~> stringLit ~
-      (DIMENSION ~> FROM ~> repsep(tableFileMapping, ",")).? ~
-      (opt(OVERWRITE) ~> INTO ~> CUBE ~> (ident <~ ".").? ~ ident) ~
-      ((PARTITIONDATA | OPTIONS) ~> "(" ~> repsep(partitionOptions, ",") <~ ")") ~
-      (FIELDS ~> TERMINATED ~> BY ~> stringLit).? <~ opt(";") ^^ {
-      case filePath ~ dimFolderPath ~ cube ~ partionDataOptions ~ delimiter =>
-        val (schema, cubename) = cube match {
-          case schemaName ~ cubeName => (schemaName, cubeName.toLowerCase())
-
-        }
-        val patitionOptionsMap = partionDataOptions.toMap
-        LoadCube(schema, cubename, filePath, dimFolderPath.getOrElse(Seq()),
-            patitionOptionsMap, false)
-    }
 
   protected lazy val loadDataNew: Parser[LogicalPlan] =
     LOAD ~> DATA ~> opt(LOCAL) ~> INPATH ~> stringLit ~ opt(OVERWRITE) ~
@@ -1129,12 +966,6 @@ class CarbonSqlParser()
       case _ => ("", "")
     }
 
-  protected lazy val showAggregateTables: Parser[LogicalPlan] =
-    SHOW ~> AGGREGATE ~> TABLES ~> (IN ~> ident).? <~ opt(";") ^^ {
-      case schema =>
-        ShowAggregateTablesCommand(schema)
-    }
-
   protected lazy val showCube: Parser[LogicalPlan] =
     SHOW ~> CUBES ~> (IN ~> ident).? ~ (DETAIL).? <~ opt(";") ^^ {
       case schema ~ detail =>
@@ -1149,53 +980,6 @@ class CarbonSqlParser()
       case _ => ShowAllCubeCommand()
     }
 
-  protected lazy val dropCubeOrTable: Parser[LogicalPlan] =
-    DROP ~> (CUBE | (AGGREGATE ~ TABLE)) ~ (IF ~> EXISTS).? ~ (ident <~ ".").? ~ ident <~
-      opt(";") ^^ {
-      case tabletype ~ exists ~ schemaName ~ resourceName =>
-        tabletype match {
-          case agg ~ table =>
-            DropAggregateTableCommand(exists.isDefined, schemaName, resourceName.toLowerCase())
-          case _ => DropCubeCommand(exists.isDefined, schemaName, resourceName.toLowerCase())
-        }
-    }
-
-  protected lazy val cubeRelation: Parser[Relation] =
-    RELATION ~> "(" ~> FACT ~> ("." ~> ident) ~ ("=" ~> ident) <~ ")" ^^ {
-      case lcol ~ rcol => Relation(lcol, rcol)
-    }
-
-  protected lazy val simpleCubeRelation: Parser[Relation] =
-    RELATION ~> "(" ~> FACT ~> ("." ~> ident) ~ ("=" ~> ident) <~ ")" ^^ {
-      case lcol ~ rcol => Relation(lcol, rcol)
-    }
-
-  protected lazy val colsFilter: Parser[FilterCols] =
-    (INCLUDE | EXCLUDE) ~ ("(" ~> repsep(ident | stringLit, ",") <~ ")") ^^ {
-      case includeKey ~ fieldList => FilterCols(includeKey, fieldList)
-    }
-
-  protected lazy val dimRelation: Parser[DimensionRelation] =
-    (ident <~ ":") ~ (dbTableIdentifier | stringLit) ~ cubeRelation ~ ((INCLUDE | EXCLUDE) ~ ("(" ~>
-      repsep(ident | stringLit, ",") <~ ")")).? ^^ {
-      case tableName ~ dimSource ~ relation ~ filterCols =>
-        val (includeKey, fieldList) = filterCols match {
-          case Some(includeKey ~ fieldList) => (includeKey, fieldList)
-          case others => ("", Seq())
-        }
-        DimensionRelation(tableName, dimSource, relation, Some(includeKey), Some(fieldList))
-    }
-
-  protected lazy val dimRelations: Parser[Seq[DimensionRelation]] = repsep(dimRelation, ",")
-
-  protected lazy val simpleDimRelation: Parser[DimensionRelation] =
-    ident ~ simpleCubeRelation ~ (INCLUDE ~> ("(" ~> repsep(ident | stringLit, ",") <~ ")")) ^^ {
-      case tableName ~ relation ~ colList =>
-        DimensionRelation(tableName, "", relation, Some("INCLUDE"), Some(colList))
-    }
-
-  protected lazy val simpleDimRelations: Parser[Seq[DimensionRelation]] = repsep(simpleDimRelation,
-    ",")
 
   protected lazy val dimCol: Parser[Field] = anyFieldDef
 
@@ -1395,7 +1179,7 @@ class CarbonSqlParser()
   }
 
   protected lazy val showLoads: Parser[LogicalPlan] =
-    SHOW ~> (LOADS|SEGMENTS) ~> FOR ~> (CUBE | TABLE) ~> (ident <~ ".").? ~ ident ~
+    SHOW ~> (LOADS|SEGMENTS) ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident ~
       (LIMIT ~> numericLit).? <~
       opt(";") ^^ {
       case schemaName ~ cubeName ~ limit =>
@@ -1408,7 +1192,7 @@ class CarbonSqlParser()
       )
 
   protected lazy val deleteLoadsByID: Parser[LogicalPlan] =
-    DELETE ~> (LOAD|SEGMENT) ~> repsep(segmentId, ",") ~ (FROM ~> (CUBE | TABLE) ~>
+    DELETE ~> (LOAD|SEGMENT) ~> repsep(segmentId, ",") ~ (FROM ~> TABLE ~>
       (ident <~ ".").? ~ ident) <~
       opt(";") ^^ {
       case loadids ~ cube => cube match {
@@ -1416,9 +1200,8 @@ class CarbonSqlParser()
       }
     }
 
-  @deprecated
   protected lazy val deleteLoadsByLoadDate: Parser[LogicalPlan] =
-    DELETE ~> (LOADS|SEGMENTS) ~> FROM ~> (CUBE | TABLE) ~> (ident <~ ".").? ~ ident ~
+    DELETE ~> (LOADS|SEGMENTS) ~> FROM ~> TABLE ~> (ident <~ ".").? ~ ident ~
       (WHERE ~> (STARTTIME <~ BEFORE) ~ stringLit) <~
       opt(";") ^^ {
       case schema ~ cube ~ condition =>
@@ -1428,18 +1211,8 @@ class CarbonSqlParser()
         }
     }
 
-  protected lazy val deleteLoadsByDate: Parser[LogicalPlan] =
-    DELETE ~> FROM ~> CUBE ~> (ident <~ ".").? ~ ident ~ (WHERE ~> (ident <~ BEFORE) ~ stringLit) <~
-      opt(";") ^^ {
-      case schema ~ cube ~ condition =>
-        condition match {
-          case dateField ~ dateValue =>
-            DeleteLoadByDate(schema, cube.toLowerCase(), dateField, dateValue)
-        }
-    }
-
   protected lazy val cleanFiles: Parser[LogicalPlan] =
-    CLEAN ~> FILES ~> FOR ~> (CUBE | TABLE) ~> (ident <~ ".").? ~ ident <~ opt(";") ^^ {
+    CLEAN ~> FILES ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident <~ opt(";") ^^ {
       case schemaName ~ cubeName => CleanFiles(schemaName, cubeName.toLowerCase())
     }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
index 9405a7f..f5fc8e9 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionCardinalityBoundryTest.scala
@@ -41,12 +41,12 @@ class DataCompactionCardinalityBoundryTest extends QueryTest with BeforeAndAfter
     var csvFilePath3 = currentDirectory + "/src/test/resources/compaction/compaction3.csv"
 
 
-    sql("LOAD DATA fact from '" + csvFilePath1 + "' INTO CUBE cardinalityTest PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath1 + "' INTO TABLE cardinalityTest OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     CarbonProperties.getInstance().addProperty("carbon.enable.load.merge", "true")
-    sql("LOAD DATA fact from '" + csvFilePath2 + "' INTO CUBE cardinalityTest  PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath2 + "' INTO TABLE cardinalityTest  OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     CarbonProperties.getInstance().addProperty("carbon.enable.load.merge", "true")
     System.out
@@ -54,8 +54,8 @@ class DataCompactionCardinalityBoundryTest extends QueryTest with BeforeAndAfter
         .getProperty("carbon.enable.load.merge")
       )
     // compaction will happen here.
-    sql("LOAD DATA fact from '" + csvFilePath3 + "' INTO CUBE cardinalityTest  PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath3 + "' INTO TABLE cardinalityTest  OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     // compaction will happen here.
     sql("alter table cardinalityTest compact 'major'"

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
index 27fbb1a..e121214 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
@@ -55,17 +55,17 @@ class DataCompactionLockTest extends QueryTest with BeforeAndAfterAll {
     var csvFilePath2 = currentDirectory + "/src/test/resources/compaction/compaction2.csv"
     var csvFilePath3 = currentDirectory + "/src/test/resources/compaction/compaction3.csv"
 
-    sql("LOAD DATA fact from '" + csvFilePath1 + "' INTO CUBE compactionLockTestTable " +
-      "PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath1 + "' INTO TABLE compactionLockTestTable " +
+      "OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
-    sql("LOAD DATA fact from '" + csvFilePath2 + "' INTO CUBE compactionLockTestTable  " +
-      "PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath2 + "' INTO TABLE compactionLockTestTable  " +
+      "OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
-    sql("LOAD DATA fact from '" + csvFilePath3 + "' INTO CUBE compactionLockTestTable  " +
-      "PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath3 + "' INTO TABLE compactionLockTestTable  " +
+      "OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     // take the lock so that next compaction will be failed.
     carbonLock.lockWithRetries()

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
index 1a7d9ba..9f87ada 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionTest.scala
@@ -39,12 +39,12 @@ class DataCompactionTest extends QueryTest with BeforeAndAfterAll {
     var csvFilePath2 = currentDirectory + "/src/test/resources/compaction/compaction2.csv"
     var csvFilePath3 = currentDirectory + "/src/test/resources/compaction/compaction3.csv"
 
-    sql("LOAD DATA fact from '" + csvFilePath1 + "' INTO CUBE normalcompaction PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath1 + "' INTO TABLE normalcompaction OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     CarbonProperties.getInstance().addProperty("carbon.enable.load.merge", "true")
-    sql("LOAD DATA fact from '" + csvFilePath2 + "' INTO CUBE normalcompaction  PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath2 + "' INTO TABLE normalcompaction  OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     CarbonProperties.getInstance().addProperty("carbon.enable.load.merge", "true")
     System.out
@@ -52,8 +52,8 @@ class DataCompactionTest extends QueryTest with BeforeAndAfterAll {
         .getProperty("carbon.enable.load.merge")
       )
     // compaction will happen here.
-    sql("LOAD DATA fact from '" + csvFilePath3 + "' INTO CUBE normalcompaction  PARTITIONDATA" +
-      "(DELIMITER ',', QUOTECHAR '\"')"
+    sql("LOAD DATA LOCAL INPATH '" + csvFilePath3 + "' INTO TABLE normalcompaction  OPTIONS" +
+      "('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
     )
     // compaction will happen here.
     sql("alter table normalcompaction compact 'major'"

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
index a357b31..44195d2 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
@@ -49,38 +49,37 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
         "NO_DICTIONARY_HIVE_6"
     );
     //For Carbon cube creation.
-    sql("CREATE CUBE NO_DICTIONARY_CARBON_6 DIMENSIONS (empno Integer, " +
-      "doj Timestamp, workgroupcategory Integer, empname String,workgroupcategoryname String, " +
-      "deptno Integer, deptname String, projectcode Integer, projectjoindate Timestamp, " +
-      "projectenddate Timestamp, designation String) MEASURES (attendance Integer,utilization " +
-      "Integer,salary Integer) " + "OPTIONS (NO_DICTIONARY(empno,empname,designation) PARTITIONER" +
-      " [PARTITION_COUNT=1])"
-    ).show()
+    sql("CREATE TABLE NO_DICTIONARY_CARBON_6 (empno Int, " +
+      "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
+      "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
+      "projectenddate Timestamp, designation String,attendance Int,utilization " +
+      "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
+        "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno,empname,designation')"
+    )
     sql(
-      "LOAD DATA fact from './src/test/resources/data.csv' INTO CUBE NO_DICTIONARY_CARBON_6 " +
-        "PARTITIONDATA(DELIMITER ',', QUOTECHAR '\"')"
-    );
+      "LOAD DATA LOCAL INPATH './src/test/resources/data.csv' INTO TABLE NO_DICTIONARY_CARBON_6 " +
+        "OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
+    )
 
-    sql("CREATE CUBE NO_DICTIONARY_CARBON_7 DIMENSIONS (empno string, " +
-      "doj Timestamp, workgroupcategory Integer, empname String,workgroupcategoryname String, " +
-      "deptno Integer, deptname String, projectcode Integer, projectjoindate Timestamp, " +
-      "projectenddate Timestamp, designation String) MEASURES (attendance Integer,utilization " +
-      "Integer,salary Integer) " + "OPTIONS (NO_DICTIONARY(empno,empname,designation) PARTITIONER" +
-      " [PARTITION_COUNT=1])"
-    ).show()
+    sql("CREATE TABLE NO_DICTIONARY_CARBON_7 (empno string, " +
+      "doj Timestamp, workgroupcategory Int, empname String,workgroupcategoryname String, " +
+      "deptno Int, deptname String, projectcode Int, projectjoindate Timestamp, " +
+      "projectenddate Timestamp, designation String,attendance Int,utilization " +
+      "Int,salary Int) STORED BY 'org.apache.carbondata.format' " +
+      "TBLPROPERTIES('DICTIONARY_EXCLUDE'='empno,empname,designation')"
+    )
     sql(
-      "LOAD DATA fact from './src/test/resources/data.csv' INTO CUBE NO_DICTIONARY_CARBON_7 " +
-        "PARTITIONDATA(DELIMITER ',', QUOTECHAR '\"')"
-    );
-    sql("CREATE CUBE filtertestTable DIMENSIONS (ID Integer,date Timestamp, country String, " +
-      "name String, phonetype String, serialname String) " +
-      "MEASURES (salary Integer) " +
-      "OPTIONS (NO_DICTIONARY(ID) PARTITIONER [PARTITION_COUNT=1])"
-    ).show()
+      "LOAD DATA LOCAL INPATH './src/test/resources/data.csv' INTO TABLE NO_DICTIONARY_CARBON_7 " +
+      "OPTIONS('DELIMITER'= ',', 'QUOTECHAR'= '\"')"
+    )
+    sql("CREATE TABLE filtertestTable (ID Int,date Timestamp, country String, " +
+      "name String, phonetype String, serialname String, salary Int) " +
+        "STORED BY 'org.apache.carbondata.format' " +  "TBLPROPERTIES('DICTIONARY_EXCLUDE'='ID')"
+    )
     sql(
-      s"LOAD DATA FACT FROM './src/test/resources/data2.csv' INTO CUBE filtertestTable OPTIONS" +
-        s"(DELIMITER ',', " +
-        s"FILEHEADER '')"
+      s"LOAD DATA LOCAL INPATH './src/test/resources/data2.csv' INTO TABLE filtertestTable OPTIONS"+
+        s"('DELIMITER'= ',', " +
+        s"'FILEHEADER'= '')"
     );
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
index cd34c97..afbda24 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
@@ -177,7 +177,7 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
   }
 
   override def afterAll {
-    sql("drop cube noloadtable")
+    sql("drop table noloadtable")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/NullMeasureValueTestCaseFilter.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/NullMeasureValueTestCaseFilter.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/NullMeasureValueTestCaseFilter.scala
index 9fcd3fd..2449f64 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/NullMeasureValueTestCaseFilter.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/NullMeasureValueTestCaseFilter.scala
@@ -32,7 +32,7 @@ class NullMeasureValueTestCaseFilter extends QueryTest with BeforeAndAfterAll {
   }
 
   override def afterAll {
-    sql("drop cube t3")
+    sql("drop table t3")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/9d846e41/integration/spark/src/test/scala/org/carbondata/spark/testsuite/measurenullvalue/NullMeasureValueTestCaseAggregate.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/measurenullvalue/NullMeasureValueTestCaseAggregate.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/measurenullvalue/NullMeasureValueTestCaseAggregate.scala
index eff75b2..6a28447 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/measurenullvalue/NullMeasureValueTestCaseAggregate.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/measurenullvalue/NullMeasureValueTestCaseAggregate.scala
@@ -59,7 +59,7 @@ class NullMeasureValueTestCaseAggregate extends QueryTest with BeforeAndAfterAll
    }
    
   override def afterAll {
-    sql("drop cube t3")
+    sql("drop table t3")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
   }


[21/50] [abbrv] incubator-carbondata git commit: [Issue-643] Column Property addition, extract interface for dictionary (#641)

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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..196d677 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
@@ -35,6 +35,7 @@ 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.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -241,17 +242,21 @@ public class FileStoreSurrogateKeyGenForCSV extends CarbonCSVBasedDimSurrogateKe
         List<GenericDataType> primitiveChild = new ArrayList<GenericDataType>();
         complexType.getAllPrimitiveChildren(primitiveChild);
         for (GenericDataType eachPrimitive : primitiveChild) {
+          ColumnIdentifier columnIdentifier = new ColumnIdentifier(eachPrimitive.getColumnId(),
+              columnsInfo.getColumnProperties(eachPrimitive.getName()), details.getColumnType());
           String dimColumnName =
               tableName + CarbonCommonConstants.UNDERSCORE + eachPrimitive.getName();
           DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-              new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
-                  eachPrimitive.getColumnId());
+              new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
           dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
           dictionaryKeys.add(dimColumnName);
         }
       } else {
+        ColumnIdentifier columnIdentifier =
+            new ColumnIdentifier(dimColumnIds[i], columnsInfo.getColumnProperties(dimColName),
+                details.getColumnType());
         DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-            new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, dimColumnIds[i]);
+            new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
         dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
         dictionaryKeys.add(dimColumnNames[i]);
       }
@@ -386,4 +391,3 @@ public class FileStoreSurrogateKeyGenForCSV extends CarbonCSVBasedDimSurrogateKe
   }
 
 }
-

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/processing/src/main/java/org/carbondata/processing/util/CarbonSchemaParser.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/util/CarbonSchemaParser.java b/processing/src/main/java/org/carbondata/processing/util/CarbonSchemaParser.java
index a6a9328..3a789f1 100644
--- a/processing/src/main/java/org/carbondata/processing/util/CarbonSchemaParser.java
+++ b/processing/src/main/java/org/carbondata/processing/util/CarbonSchemaParser.java
@@ -20,6 +20,7 @@
 package org.carbondata.processing.util;
 
 import java.util.*;
+import java.util.Map.Entry;
 
 import org.carbondata.core.carbon.CarbonDataLoadSchema;
 import org.carbondata.core.carbon.CarbonDataLoadSchema.DimensionRelation;
@@ -1229,4 +1230,47 @@ public final class CarbonSchemaParser {
     }
     return counter;
   }
+
+  public static String getColumnPropertiesString(List<CarbonDimension> dimensions) {
+    StringBuilder colPropertiesString = new StringBuilder();
+    for (int dim = 0; dim < dimensions.size(); dim++) {
+      CarbonDimension dimension = dimensions.get(dim);
+      if (dimension.isComplex()) {
+        List<CarbonDimension> childs = dimension.getListOfChildDimensions();
+        for (CarbonDimension child : childs) {
+          buildDimensionColumnPropertyString(child, colPropertiesString, dim);
+        }
+      } else {
+        buildDimensionColumnPropertyString(dimension, colPropertiesString, dim);
+      }
+
+    }
+
+    return colPropertiesString.toString();
+  }
+
+  protected static void buildDimensionColumnPropertyString(CarbonDimension dimension,
+      StringBuilder colPropertiesString, int dim) {
+    Map<String, String> columnProperties = dimension.getColumnProperties();
+    if (null != columnProperties && columnProperties.size() > 0) {
+      if (colPropertiesString.length() > 0) {
+        colPropertiesString.append(CarbonCommonConstants.HASH_SPC_CHARACTER);
+      }
+      colPropertiesString.append(dimension.getColName())
+          .append(CarbonCommonConstants.COLON_SPC_CHARACTER);
+      int size = columnProperties.entrySet().size();
+      int index = 0;
+      Iterator<Entry<String, String>> itr = columnProperties.entrySet().iterator();
+      while (itr.hasNext()) {
+        Entry<String, String> entry = itr.next();
+        colPropertiesString.append(entry.getKey())
+            .append(CarbonCommonConstants.HYPHEN_SPC_CHARACTER);
+        colPropertiesString.append(entry.getValue());
+        index++;
+        if (index < size) {
+          colPropertiesString.append(CarbonCommonConstants.COMA_SPC_CHARACTER);
+        }
+      }
+    }
+  }
 }


[36/50] [abbrv] incubator-carbondata git commit: [BUG] Timestamp column filter is not working for unknown filter (#779)

Posted by ch...@apache.org.
[BUG] Timestamp column filter is not working for unknown filter (#779)



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

Branch: refs/heads/master
Commit: 697ba7b7e38ff0aa8966ef2bfbac15b5435e9835
Parents: 114b471
Author: nareshpr <pr...@gmail.com>
Authored: Thu Jun 30 00:10:19 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Thu Jun 30 00:10:19 2016 +0530

----------------------------------------------------------------------
 .../filters/FilterExpressionProcessor.java      | 21 ++++++++------------
 1 file changed, 8 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/697ba7b7/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
index dded529..8a85599 100644
--- a/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java
@@ -302,7 +302,8 @@ public class FilterExpressionProcessor implements FilterProcessor {
             && currentCondExpression.getColumnList().get(0).getCarbonColumn().getDataType()
             != DataType.STRUCT) {
           if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
-              .hasEncoding(Encoding.DICTIONARY)) {
+              .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())
@@ -333,18 +334,12 @@ public class FilterExpressionProcessor implements FilterProcessor {
             && 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);
-            }
+          if (condExpression.getColumnList().get(0).getCarbonColumn()
+              .hasEncoding(Encoding.DICTIONARY) && !condExpression.getColumnList().get(0)
+              .getCarbonColumn().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+            return new ConditionalFilterResolverImpl(expression, true, true);
+          } else {
+            return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
           }
         } else {
           return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);


[42/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
index bbc9f25,9f534c1..4d39eb2
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
@@@ -52,16 -51,14 +52,18 @@@ class CarbonSourc
    override def createRelation(
        sqlContext: SQLContext,
        parameters: Map[String, String]): BaseRelation = {
-     parameters.get("path") match {
-       case Some(path) => CarbonDatasourceHadoopRelation(sqlContext, Array(path), parameters)
-       case _ =>
-         val options = new CarbonOption(parameters)
-         val tableIdentifier = options.tableIdentifier.split("""\.""").toSeq
-         val ident = tableIdentifier match {
-           case Seq(name) => TableIdentifier(name, None)
-           case Seq(db, name) => TableIdentifier(name, Some(db))
-         }
-         CarbonDatasourceRelation(ident, None)(sqlContext)
+     if (parameters.get("tablePath") != None) {
+       val options = new CarbonOption(parameters)
+       val tableIdentifier = options.tableIdentifier.split("""\.""").toSeq
 -      CarbonDatasourceRelation(tableIdentifier, None)(sqlContext)
++      val ident = tableIdentifier match {
++        case Seq(name) => TableIdentifier(name, None)
++        case Seq(db, name) => TableIdentifier(name, Some(db))
++      }
++      CarbonDatasourceRelation(ident, None)(sqlContext)
+     } else if (parameters.get("path") != None) {
+       CarbonDatasourceHadoopRelation(sqlContext, Array(parameters.get("path").get), parameters)
+     } else {
+       sys.error("Carbon table path not found")
      }
  
    }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 05149eb,ce43c4f..8a9f1c9
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@@ -29,11 -29,10 +29,11 @@@ import org.apache.spark.unsafe.types.UT
  
  import org.carbondata.core.cache.{Cache, CacheProvider, CacheType}
  import org.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
- import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+ import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
  import org.carbondata.core.carbon.metadata.datatype.DataType
  import org.carbondata.core.carbon.metadata.encoder.Encoding
 -import org.carbondata.query.carbon.util.DataTypeUtil
 +import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 +import org.carbondata.scan.util.DataTypeUtil
  
  /**
   * It decodes the data.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index 4209c5a,b340884..406b025
--- 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,13 -1186,12 +1186,13 @@@ 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) ~>
+     DELETE ~> (LOAD|SEGMENT) ~> repsep(segmentId, ",") ~ (FROM ~> TABLE ~>
        (ident <~ ".").? ~ ident) <~
        opt(";") ^^ {
        case loadids ~ cube => cube match {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/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 cb94f72,75abe0e..b8afcdf
--- 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
@@@ -29,8 -29,10 +29,11 @@@ import scala.util.Rando
  
  import org.apache.spark.SparkEnv
  import org.apache.spark.sql._
 +import org.apache.spark.sql.catalyst.TableIdentifier
+ import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
 -import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal}
 +import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+ import org.apache.spark.sql.catalyst.util.DateTimeUtils
+ import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp
  import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
  import org.apache.spark.sql.hive.HiveContext
  import org.apache.spark.sql.types.TimestampType
@@@ -1244,7 -1269,8 +1266,8 @@@ private[sql] case class CreateCube(cm: 
        try {
          sqlContext.sql(
            s"""CREATE TABLE $dbName.$tbName USING org.apache.spark.sql.CarbonSource""" +
-           s""" OPTIONS (tableName "$dbName.$tbName", tablePath "$cubePath") """).collect
 -          s""" OPTIONS (cubename "$dbName.$tbName", tablePath "$tablePath") """)
++          s""" OPTIONS (tableName "$dbName.$tbName", tablePath "$tablePath") """)
+               .collect
        } catch {
          case e: Exception =>
  
@@@ -1290,21 -1317,23 +1314,21 @@@ private[sql] case class DeleteLoadsById
  
      // validate load ids first
      validateLoadIds
 +    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 $dbName.$tableName does not exist")
 -      LOGGER.audit(s"Delete load by Id is failed. Table $schemaName.$tableName does not exist")
 -      sys.error(s"Table $schemaName.$tableName does not exist")
++      LOGGER.audit(s"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
  
@@@ -1356,20 -1375,29 +1370,26 @@@ private[sql] case class DeleteLoadsByLo
  
    def run(sqlContext: SQLContext): Seq[Row] = {
  
 -    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 -    LOGGER.audit(s"Delete load by load date request has been received for $schemaName.$tableName")
 -
 -    val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog.lookupRelation1(
 -      Option(schemaName),
 -      tableName,
 -     None
 -    )(sqlContext).asInstanceOf[CarbonRelation]
 +    LOGGER.audit("The delete load by load date request has been received.")
 +    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 $dbName.$tableName does not " +
 -        .audit(s"Delete load by load date is failed. Table $schemaName.$tableName does not " +
++        .audit(s"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")
      }
  
+     val timeObj = Cast(Literal(loadDate), TimestampType).eval()
+     if(null == timeObj) {
+       val errorMessage = "Error: Invalid load start time format " + loadDate
+       throw new MalformedCarbonCommandException(errorMessage)
+     }
+ 
      var carbonTable = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
 -      .getCarbonTable(schemaName + '_' + tableName)
 +      .getCarbonTable(dbName + '_' + tableName)
      var segmentStatusManager = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier)
  
      if (null == carbonTable) {
@@@ -1378,9 -1409,14 +1398,14 @@@
      }
      var path = carbonTable.getMetaDataFilepath()
  
- 
-     var invalidLoadTimestamps = segmentStatusManager.updateDeletionStatus(loadDate, path).asScala
-     LOGGER.audit("The delete load by Id is successfull.")
+     var invalidLoadTimestamps = segmentStatusManager
+       .updateDeletionStatus(loadDate, path, timeObj.asInstanceOf[java.lang.Long]).asScala
+     if(invalidLoadTimestamps.isEmpty) {
 -      LOGGER.audit(s"Delete load by load date is successfull for $schemaName.$tableName.")
++      LOGGER.audit(s"Delete load by load date is successfull for $dbName.$tableName.")
+     }
+     else {
+       sys.error("Delete load by load date is failed. No matching load found.")
+     }
      Seq.empty
  
    }
@@@ -1742,9 -1818,7 +1768,7 @@@ private[sql] case class DropTableComman
          if (carbonLock.lockWithRetries()) {
            logInfo("Successfully able to get the table metadata file lock")
          } else {
-           LOGGER.audit(
-             s"Dropping table with Database name [$dbName] and Table name [$tableName] " +
-             s"failed as the Table is locked")
 -          LOGGER.audit(s"Dropping table $schemaName.$cubeName failed as the Table is locked")
++          LOGGER.audit(s"Dropping table $dbName.$tableName failed as the Table is locked")
            sys.error("Table is locked for updation. Please try after some time")
          }
  
@@@ -1877,7 -1976,9 +1901,9 @@@ private[sql] case class DescribeCommand
  
    override def run(sqlContext: SQLContext): Seq[Row] = {
      val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
 -      .lookupRelation2(tblIdentifier, None)(sqlContext).asInstanceOf[CarbonRelation]
 +      .lookupRelation1(tblIdentifier)(sqlContext).asInstanceOf[CarbonRelation]
+     val mapper = new ObjectMapper()
+     val colProps = StringBuilder.newBuilder
      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(
@@@ -1966,17 -2085,16 +2010,17 @@@ private[sql] case class DeleteLoadByDat
  
    def run(sqlContext: SQLContext): Seq[Row] = {
  
-     LOGGER.audit("The delete load by date request has been received.")
 -    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 -    LOGGER.audit(s"The delete load by date request has been received for $schemaName.$cubeName")
 +    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
++    LOGGER.audit(s"The delete load by date request has been received for $dbName.$cubeName")
 +    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(
@@@ -1986,8 -2104,8 +2030,8 @@@
      if (matches.isEmpty) {
        LOGGER.audit(
          "The delete load by date is failed. " +
-         "Table $dbName.$cubeName does not contain date field " + dateField)
 -        s"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
@@@ -2007,7 -2125,7 +2051,7 @@@
        actualColName,
        dateValue,
        relation.cubeMeta.partitioner)
-     LOGGER.audit("The delete load by date is successfull.")
 -    LOGGER.audit(s"The delete load by date $dateValue is successful for $schemaName.$cubeName.")
++    LOGGER.audit(s"The delete load by date $dateValue is successful for $dbName.$cubeName.")
      Seq.empty
    }
  }
@@@ -2020,15 -2138,14 +2064,15 @@@ private[sql] case class CleanFiles
    val LOGGER = LogServiceFactory.getLogService(this.getClass.getCanonicalName)
  
    def run(sqlContext: SQLContext): Seq[Row] = {
-     LOGGER.audit("The clean files request has been received.")
 -    val schemaName = getDB.getDatabaseName(schemaNameOp, sqlContext)
 -    LOGGER.audit(s"Clean files request has been received for $schemaName.$cubeName")
 +    val dbName = getDB.getDatabaseName(schemaNameOp, sqlContext)
++    LOGGER.audit(s"The clean files request has been received for $dbName.$cubeName")
 +    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"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()

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
index d87e132,f88b74e..aeae761
--- 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
@@@ -113,14 -112,30 +113,6 @@@ class CarbonMetastoreCatalog(hive: Hive
  
    val metadata = loadMetadata(storePath)
  
-   lazy val useUniquePath = if ("true".equalsIgnoreCase(CarbonProperties.getInstance().
-     getProperty(
-       CarbonCommonConstants.CARBON_UNIFIED_STORE_PATH,
-       CarbonCommonConstants.CARBON_UNIFIED_STORE_PATH_DEFAULT))) {
-     true
-   } else {
-     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],
 -      alias: Option[String] = None): LogicalPlan = {
 -    try {
 -      super.lookupRelation(tableIdentifier, alias)
 -    } catch {
 -      case s: java.lang.Exception =>
 -        lookupRelation2(tableIdentifier, alias)(hive.asInstanceOf[SQLContext])
 -    }
--  }
  
    def getCubeCreationTime(schemaName: String, cubeName: String): Long = {
      val cubeMeta = metadata.cubesMeta.filter(

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonDataRDDFactory.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
----------------------------------------------------------------------
diff --cc integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
index 0de2d1a,0000000..e7f131d
mode 100644,000000..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
@@@ -1,218 -1,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.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
 +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.hadoop.{CarbonInputFormat, CarbonInputSplit}
 +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
 +
 +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[V: ClassTag](
 +  sc: SparkContext,
 +  queryModel: QueryModel,
 +  filterExpression: Expression,
 +  keyClass: RawValue[V],
 +  @transient conf: Configuration,
 +  cubeCreationTime: Long,
 +  schemaLastUpdatedTime: Long,
 +  baseStoreLocation: String)
 +  extends RDD[V](sc, Nil) with Logging {
 +
 +  val defaultParallelism = sc.defaultParallelism
 +
 +  override def getPartitions: Array[Partition] = {
 +    val startTime = System.currentTimeMillis()
 +    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 {
-       val filterResolver = carbonInputFormat
-         .getResolvedFilter(job.getConfiguration, filterExpression)
- 
-       CarbonInputFormat.setFilterPredicates(job.getConfiguration, filterResolver)
-       queryModel.setFilterExpressionResolverTree(filterResolver)
++      // before applying filter check whether segments are available in the table.
++      val splits = carbonInputFormat.getSplits(job)
++      if (!splits.isEmpty) {
++        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[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)
 +           // 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 ChunkRowIterator(
 +             QueryExecutorFactory.getQueryExecutor(queryModel).execute(queryModel).
 +               asInstanceOf[CarbonIterator[BatchResult]]).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(): V = {
 +         if (!hasNext) {
 +           throw new java.util.NoSuchElementException("End of stream")
 +         }
 +         havePair = false
 +         keyClass.getValue(rowIterator.next())
 +       }
 +
 +       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/7f722186/integration/spark/src/main/scala/org/carbondata/spark/util/GlobalDictionaryUtil.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/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 b6bda0c,4a02975..d13a9df
--- 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,19 -40,15 +40,19 @@@ class AllDataTypesTestCaseAggregate ext
      val currentDirectory = new File(this.getClass.getResource("/").getPath + "/../../")
        .getCanonicalPath
  
-     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("create table Carbon_automation_test (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_phonePADPart
 itionedVersions 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) STORED BY 'org.apache.carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Latest_MONTH,Latest_DAY,deviceInformationId')");
      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("LOAD DATA LOCAL INPATH '"+currentDirectory+"/src/test/resources/100_olap.csv' INTO table Carbon_automation_test options('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,Lat
 est_EMUIVersion,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")
+     sql("drop table Carbon_automation_test")
++    sql("drop table Carbon_automation_hive")
 +
      CarbonProperties.getInstance()
        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
    }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/test/scala/org/carbondata/spark/util/GlobalDictionaryUtilTestCase.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java
index 0a0fe62,0000000..a1cf40a
mode 100644,000000..100644
--- a/processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java
+++ b/processing/src/main/java/org/carbondata/lcm/locks/LocalFileLock.java
@@@ -1,162 -1,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('/'));
++    cubeName = tempStr.substring(tempStr.lastIndexOf('/') + 1, tempStr.length());
++    tempStr = tempStr.substring(0, tempStr.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/7f722186/processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java
index e3eab01,0000000..3b03b1f
mode 100644,000000..100644
--- a/processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java
+++ b/processing/src/main/java/org/carbondata/lcm/locks/LockUsage.java
@@@ -1,29 -1,0 +1,30 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + */
 +package org.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;
++  COMPACTION_LOCK,
++  TABLE_STATUS_LOCK;
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
----------------------------------------------------------------------
diff --cc processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
index dcd5e19,473840b..3dd64be
--- a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@@ -37,9 -39,10 +39,8 @@@ import org.carbondata.core.carbon.metad
  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;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerColumnar.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/processing/src/main/java/org/carbondata/processing/store/writer/CarbonFactDataWriterImplForIntIndexAndAggBlock.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenStep.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/FileStoreSurrogateKeyGenForCSV.java
----------------------------------------------------------------------



[23/50] [abbrv] incubator-carbondata git commit: [Issue-643] Column Property addition, extract interface for dictionary (#641)

Posted by ch...@apache.org.
[Issue-643] Column Property addition, extract interface for dictionary (#641)

1.Added interface to get dictionary service
2.Modified schema.thrift to add column properties
3. Describe formatted shows column properties

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

Branch: refs/heads/master
Commit: 38d84e0e2d89744290d50d46750bbadf5ba9c693
Parents: 5f86244
Author: ashokblend <as...@gmail.com>
Authored: Mon Jun 27 16:41:52 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Mon Jun 27 16:41:52 2016 +0530

----------------------------------------------------------------------
 .../common/ext/ColumnUniqueIdGenerator.java     |  41 +++++++
 .../common/ext/DictionaryFactory.java           | 119 +++++++++++++++++++
 .../org/carbondata/common/ext/PathFactory.java  |  48 ++++++++
 .../common/factory/CarbonCommonFactory.java     |  54 +++++++++
 .../dictionary/AbstractDictionaryCache.java     |  43 ++++---
 .../cache/dictionary/DictionaryCacheLoader.java |   7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |  28 ++---
 .../DictionaryColumnUniqueIdentifier.java       |   9 +-
 .../dictionary/ForwardDictionaryCache.java      |  11 +-
 .../dictionary/ReverseDictionaryCache.java      |  11 +-
 .../core/carbon/ColumnIdentifier.java           |  91 ++++++++++++++
 .../ThriftWrapperSchemaConverterImpl.java       |   4 +
 .../schema/table/column/CarbonColumn.java       |  23 ++++
 .../schema/table/column/ColumnSchema.java       |  49 ++++++++
 .../core/carbon/path/CarbonTablePath.java       |  20 ++--
 .../core/constants/CarbonCommonConstants.java   |   8 ++
 .../CarbonDictionaryMetadataReaderImpl.java     |   9 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |  20 +++-
 .../CarbonDictionarySortIndexReaderImpl.java    |   7 +-
 .../core/service/ColumnUniqueIdService.java     |  34 ++++++
 .../core/service/DictionaryService.java         |  92 ++++++++++++++
 .../carbondata/core/service/PathService.java    |  38 ++++++
 .../org/carbondata/core/util/CarbonUtil.java    |  64 ++++++++++
 .../core/writer/CarbonDictionaryWriterImpl.java |  41 +++++--
 .../CarbonDictionarySortIndexWriterImpl.java    |  19 +--
 .../query/carbon/executor/util/QueryUtil.java   |  10 +-
 .../filters/measurefilter/util/FilterUtil.java  |   2 +-
 .../dictionary/AbstractDictionaryCacheTest.java |   7 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |   4 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |   4 +-
 ...CarbonDictionarySortIndexReaderImplTest.java |   6 +-
 .../writer/CarbonDictionaryWriterImplTest.java  |   9 +-
 ...CarbonDictionarySortIndexWriterImplTest.java |  13 +-
 .../examples/GenerateDictionaryExample.scala    |   4 +-
 format/src/main/thrift/schema.thrift            |  13 +-
 .../AbstractDictionaryDecodedReadSupport.java   |   4 +-
 .../hadoop/test/util/StoreCreator.java          |   8 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |   3 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |   7 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  | 107 +++++++++++++----
 .../execution/command/carbonTableSchema.scala   |  74 ++++++++----
 .../spark/sql/hive/CarbonStrategies.scala       |  10 +-
 .../apache/spark/sql/hive/CarbonStrategy.scala  |   4 +-
 .../spark/CarbonColumnValidator.scala           |  36 ++++++
 .../carbondata/spark/CarbonSparkFactory.scala   |  63 ++++++++++
 .../spark/DictionaryDetailHelper.scala          |  66 ++++++++++
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |   7 +-
 .../org/carbondata/spark/util/CommonUtil.scala  |  79 +++++++++++-
 .../spark/util/GlobalDictionaryUtil.scala       |  85 +++++--------
 .../spark/load/CarbonLoaderUtilTest.java        |   4 +-
 .../validation/FileFooterValidator.java         |   6 +-
 .../apache/spark/sql/TestCarbonSqlParser.scala  |  37 +++---
 ...ataWithMalformedCarbonCommandException.scala |   4 +-
 .../ColumnPropertyValidationTestCase.scala      |  28 +++++
 .../util/GlobalDictionaryUtilTestCase.scala     |   7 +-
 .../graphgenerator/GraphGenerator.java          |   3 +
 .../configuration/GraphConfigurationInfo.java   |  10 ++
 .../processing/schema/metadata/ColumnsInfo.java |  10 ++
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      |  36 ++++++
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |   1 +
 .../FileStoreSurrogateKeyGenForCSV.java         |  12 +-
 .../processing/util/CarbonSchemaParser.java     |  44 +++++++
 62 files changed, 1455 insertions(+), 262 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java b/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
new file mode 100644
index 0000000..0ef3950
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.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.common.ext;
+
+import java.util.UUID;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.carbondata.core.service.ColumnUniqueIdService;
+
+/**
+ * It returns unique id given column
+ */
+public class ColumnUniqueIdGenerator implements ColumnUniqueIdService {
+
+  private static ColumnUniqueIdService columnUniqueIdService = new ColumnUniqueIdGenerator();
+
+  @Override public String generateUniqueId(String schemaName, ColumnSchema columnSchema) {
+    return UUID.randomUUID().toString();
+  }
+
+  public static ColumnUniqueIdService getInstance() {
+    return columnUniqueIdService;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java b/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java
new file mode 100644
index 0000000..944f772
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/ext/DictionaryFactory.java
@@ -0,0 +1,119 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.ext;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
+import org.carbondata.core.reader.CarbonDictionaryReader;
+import org.carbondata.core.reader.CarbonDictionaryReaderImpl;
+import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
+import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
+import org.carbondata.core.service.DictionaryService;
+import org.carbondata.core.writer.CarbonDictionaryWriter;
+import org.carbondata.core.writer.CarbonDictionaryWriterImpl;
+import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
+import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriterImpl;
+
+/**
+ * service to get dictionary reader and writer
+ */
+public class DictionaryFactory implements DictionaryService {
+
+  private static DictionaryService dictService = new DictionaryFactory();
+
+  /**
+   * get dictionary writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionaryWriter getDictionaryWriter(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionaryWriterImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
+  }
+
+  /**
+   * get dictionary sort index writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, columnIdentifier,
+        carbonStorePath);
+  }
+
+  /**
+   * get dictionary metadata reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionaryMetadataReader getDictionaryMetadataReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionaryMetadataReaderImpl(carbonStorePath, carbonTableIdentifier,
+        columnIdentifier);
+  }
+
+  /**
+   * get dictionary reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionaryReader getDictionaryReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionaryReaderImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
+  }
+
+  /**
+   * get dictionary sort index reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  @Override public CarbonDictionarySortIndexReader getDictionarySortIndexReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath) {
+    return new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier,
+        carbonStorePath);
+  }
+
+  public static DictionaryService getInstance() {
+    return dictService;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/ext/PathFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/ext/PathFactory.java b/core/src/main/java/org/carbondata/common/ext/PathFactory.java
new file mode 100644
index 0000000..8b64aec
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/ext/PathFactory.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.common.ext;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.carbon.path.CarbonStorePath;
+import org.carbondata.core.carbon.path.CarbonTablePath;
+import org.carbondata.core.service.PathService;
+
+/**
+ * Create helper to get path details
+ */
+public class PathFactory implements PathService {
+
+  private static PathService pathService = new PathFactory();
+
+  /**
+   * @param columnIdentifier
+   * @param storeLocation
+   * @param tableIdentifier
+   * @return store path related to tables
+   */
+  @Override public CarbonTablePath getCarbonTablePath(ColumnIdentifier columnIdentifier,
+      String storeLocation, CarbonTableIdentifier tableIdentifier) {
+    return CarbonStorePath.getCarbonTablePath(storeLocation, tableIdentifier);
+  }
+
+  public static PathService getInstance() {
+    return pathService;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java b/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java
new file mode 100644
index 0000000..dfa14f9
--- /dev/null
+++ b/core/src/main/java/org/carbondata/common/factory/CarbonCommonFactory.java
@@ -0,0 +1,54 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.factory;
+
+import org.carbondata.common.ext.ColumnUniqueIdGenerator;
+import org.carbondata.common.ext.DictionaryFactory;
+import org.carbondata.common.ext.PathFactory;
+import org.carbondata.core.service.ColumnUniqueIdService;
+import org.carbondata.core.service.DictionaryService;
+import org.carbondata.core.service.PathService;
+
+/**
+ * Interface to get services
+ */
+public class CarbonCommonFactory {
+
+  /**
+   * @return dictionary service
+   */
+  public static DictionaryService getDictionaryService() {
+    return DictionaryFactory.getInstance();
+  }
+
+  /**
+   * @return path service
+   */
+  public static PathService getPathService() {
+    return PathFactory.getInstance();
+  }
+
+  /**
+   * @return unique id generator
+   */
+  public static ColumnUniqueIdService getColumnUniqueIdGenerator() {
+    return ColumnUniqueIdGenerator.getInstance();
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java b/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
index 690f125..390bcc8 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCache.java
@@ -22,16 +22,18 @@ package org.carbondata.core.cache.dictionary;
 import java.io.IOException;
 import java.util.List;
 
+import org.carbondata.common.factory.CarbonCommonFactory;
 import org.carbondata.core.cache.Cache;
 import org.carbondata.core.cache.CacheType;
 import org.carbondata.core.cache.CarbonLRUCache;
-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.impl.FileFactory;
 import org.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
-import org.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
+import org.carbondata.core.service.DictionaryService;
+import org.carbondata.core.service.PathService;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
@@ -91,12 +93,17 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   protected boolean isFileExistsForGivenColumn(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath,
-        dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
-    String dictionaryFilePath = carbonTablePath
-        .getDictionaryFilePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier());
-    String dictionaryMetadataFilePath = carbonTablePath
-        .getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier());
+    PathService pathService = CarbonCommonFactory.getPathService();
+    CarbonTablePath carbonTablePath = pathService
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier(), carbonStorePath,
+            dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
+
+    String dictionaryFilePath =
+        carbonTablePath.getDictionaryFilePath(dictionaryColumnUniqueIdentifier
+            .getColumnIdentifier().getColumnId());
+    String dictionaryMetadataFilePath =
+        carbonTablePath.getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier
+            .getColumnIdentifier().getColumnId());
     // check if both dictionary and its metadata file exists for a given column
     return CarbonUtil.isFileExists(dictionaryFilePath) && CarbonUtil
         .isFileExists(dictionaryMetadataFilePath);
@@ -111,10 +118,11 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   protected CarbonDictionaryColumnMetaChunk readLastChunkFromDictionaryMetadataFile(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) throws IOException {
-    CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.carbonStorePath,
-            dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
-            dictionaryColumnUniqueIdentifier.getColumnIdentifier());
+    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
+    CarbonDictionaryMetadataReader columnMetadataReaderImpl = dictService
+        .getDictionaryMetadataReader(dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier(),
+            dictionaryColumnUniqueIdentifier.getColumnIdentifier(), carbonStorePath);
+
     CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
     // read metadata file
     try {
@@ -148,10 +156,13 @@ public abstract class AbstractDictionaryCache<K extends DictionaryColumnUniqueId
    */
   private CarbonFile getDictionaryMetaCarbonFile(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(carbonStorePath,
-        dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
-    String dictionaryFilePath = carbonTablePath
-        .getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier());
+    PathService pathService = CarbonCommonFactory.getPathService();
+    CarbonTablePath carbonTablePath = pathService
+        .getCarbonTablePath(dictionaryColumnUniqueIdentifier.getColumnIdentifier(), carbonStorePath,
+            dictionaryColumnUniqueIdentifier.getCarbonTableIdentifier());
+    String dictionaryFilePath =
+        carbonTablePath.getDictionaryMetaFilePath(dictionaryColumnUniqueIdentifier
+            .getColumnIdentifier().getColumnId());
     FileFactory.FileType fileType = FileFactory.getFileType(dictionaryFilePath);
     CarbonFile carbonFile = FileFactory.getCarbonFile(dictionaryFilePath, fileType);
     return carbonFile;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
index 54431a7..a28d58d 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoader.java
@@ -21,6 +21,8 @@ package org.carbondata.core.cache.dictionary;
 
 import java.io.IOException;
 
+import org.carbondata.core.carbon.ColumnIdentifier;
+
 public interface DictionaryCacheLoader {
 
   /**
@@ -37,6 +39,7 @@ public interface DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  void load(DictionaryInfo dictionaryInfo, String columnIdentifier, long dictionaryChunkStartOffset,
-      long dictionaryChunkEndOffset, boolean loadSortIndex) throws IOException;
+  void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
+      long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
+      throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
index 887d8b3..770cb07 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryCacheLoaderImpl.java
@@ -22,11 +22,12 @@ package org.carbondata.core.cache.dictionary;
 import java.io.IOException;
 import java.util.List;
 
+import org.carbondata.common.factory.CarbonCommonFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.reader.CarbonDictionaryReader;
-import org.carbondata.core.reader.CarbonDictionaryReaderImpl;
 import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
-import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReaderImpl;
+import org.carbondata.core.service.DictionaryService;
 
 /**
  * This class is responsible for loading the dictionary data for given columns
@@ -67,7 +68,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    *                                   read in memory after dictionary loading
    * @throws IOException
    */
-  @Override public void load(DictionaryInfo dictionaryInfo, String columnIdentifier,
+  @Override public void load(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier,
       long dictionaryChunkStartOffset, long dictionaryChunkEndOffset, boolean loadSortIndex)
       throws IOException {
     List<byte[]> dictionaryChunk =
@@ -87,7 +88,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @return list of dictionary value
    * @throws IOException
    */
-  private List<byte[]> load(String columnIdentifier, long startOffset, long endOffset)
+  private List<byte[]> load(ColumnIdentifier columnIdentifier, long startOffset, long endOffset)
       throws IOException {
     CarbonDictionaryReader dictionaryReader = getDictionaryReader(columnIdentifier);
     List<byte[]> dictionaryValue = null;
@@ -106,7 +107,7 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @param columnIdentifier
    * @throws IOException
    */
-  private void readSortIndexFile(DictionaryInfo dictionaryInfo, String columnIdentifier)
+  private void readSortIndexFile(DictionaryInfo dictionaryInfo, ColumnIdentifier columnIdentifier)
       throws IOException {
     CarbonDictionarySortIndexReader sortIndexReader = getSortIndexReader(columnIdentifier);
     try {
@@ -123,20 +124,19 @@ public class DictionaryCacheLoaderImpl implements DictionaryCacheLoader {
    * @param columnIdentifier unique column identifier
    * @return carbon dictionary reader instance
    */
-  private CarbonDictionaryReader getDictionaryReader(String columnIdentifier) {
-    CarbonDictionaryReader dictionaryReader =
-        new CarbonDictionaryReaderImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
-    return dictionaryReader;
+  private CarbonDictionaryReader getDictionaryReader(ColumnIdentifier columnIdentifier) {
+    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
+    return dictService
+        .getDictionaryReader(carbonTableIdentifier, columnIdentifier, carbonStorePath);
   }
 
   /**
    * @param columnIdentifier unique column identifier
    * @return sort index reader instance
    */
-  private CarbonDictionarySortIndexReader getSortIndexReader(String columnIdentifier) {
-    CarbonDictionarySortIndexReader sortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier,
-            carbonStorePath);
-    return sortIndexReader;
+  private CarbonDictionarySortIndexReader getSortIndexReader(ColumnIdentifier columnIdentifier) {
+    DictionaryService dictService = CarbonCommonFactory.getDictionaryService();
+    return dictService
+        .getDictionarySortIndexReader(carbonTableIdentifier, columnIdentifier, carbonStorePath);
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
index ff3804a..dea789f 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/DictionaryColumnUniqueIdentifier.java
@@ -20,6 +20,7 @@
 package org.carbondata.core.cache.dictionary;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 
 /**
@@ -35,7 +36,7 @@ public class DictionaryColumnUniqueIdentifier {
   /**
    * unique column id
    */
-  private String columnIdentifier;
+  private ColumnIdentifier columnIdentifier;
 
   private DataType dataType;
 
@@ -47,7 +48,7 @@ public class DictionaryColumnUniqueIdentifier {
    * @param columnIdentifier
    */
   public DictionaryColumnUniqueIdentifier(CarbonTableIdentifier carbonTableIdentifier,
-      String columnIdentifier) {
+      ColumnIdentifier columnIdentifier) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
   }
@@ -61,7 +62,7 @@ public class DictionaryColumnUniqueIdentifier {
    * @param dataType
    */
   public DictionaryColumnUniqueIdentifier(CarbonTableIdentifier carbonTableIdentifier,
-      String columnIdentifier, DataType dataType) {
+      ColumnIdentifier columnIdentifier, DataType dataType) {
     this(carbonTableIdentifier, columnIdentifier);
     this.dataType = dataType;
   }
@@ -80,7 +81,7 @@ public class DictionaryColumnUniqueIdentifier {
   /**
    * @return columnIdentifier
    */
-  public String getColumnIdentifier() {
+  public ColumnIdentifier getColumnIdentifier() {
     return columnIdentifier;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java b/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
index d90efd8..bee7714 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCache.java
@@ -131,7 +131,7 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
     Dictionary forwardDictionary = null;
     ColumnDictionaryInfo columnDictionaryInfo = (ColumnDictionaryInfo) carbonLRUCache.get(
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
             CacheType.FORWARD_DICTIONARY));
     if (null != columnDictionaryInfo) {
       forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
@@ -148,8 +148,9 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
    */
   @Override public void invalidate(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    carbonLRUCache.remove(getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
-        CacheType.FORWARD_DICTIONARY));
+    carbonLRUCache.remove(
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
+            CacheType.FORWARD_DICTIONARY));
   }
 
   /**
@@ -172,12 +173,12 @@ public class ForwardDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
           "Either dictionary or its metadata does not exist for column identifier :: "
               + dictionaryColumnUniqueIdentifier.getColumnIdentifier());
     }
-    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier();
+    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId();
     ColumnDictionaryInfo columnDictionaryInfo =
         getColumnDictionaryInfo(dictionaryColumnUniqueIdentifier, columnIdentifier);
     // load sort index file in case of forward dictionary
     checkAndLoadDictionaryData(dictionaryColumnUniqueIdentifier, columnDictionaryInfo,
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
             CacheType.FORWARD_DICTIONARY), true);
     forwardDictionary = new ForwardDictionary(columnDictionaryInfo);
     return forwardDictionary;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java b/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
index 9b3db5d..6e49183 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCache.java
@@ -132,7 +132,7 @@ public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
     Dictionary reverseDictionary = null;
     ColumnReverseDictionaryInfo columnReverseDictionaryInfo =
         (ColumnReverseDictionaryInfo) carbonLRUCache.get(
-            getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+            getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
                 CacheType.REVERSE_DICTIONARY));
     if (null != columnReverseDictionaryInfo) {
       reverseDictionary = new ReverseDictionary(columnReverseDictionaryInfo);
@@ -149,8 +149,9 @@ public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
    */
   @Override public void invalidate(
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier) {
-    carbonLRUCache.remove(getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
-        CacheType.REVERSE_DICTIONARY));
+    carbonLRUCache.remove(
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
+            CacheType.REVERSE_DICTIONARY));
   }
 
   /**
@@ -173,12 +174,12 @@ public class ReverseDictionaryCache<K extends DictionaryColumnUniqueIdentifier,
           "Either dictionary or its metadata does not exist for column identifier :: "
               + dictionaryColumnUniqueIdentifier.getColumnIdentifier());
     }
-    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier();
+    String columnIdentifier = dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId();
     ColumnReverseDictionaryInfo columnReverseDictionaryInfo =
         getColumnReverseDictionaryInfo(dictionaryColumnUniqueIdentifier, columnIdentifier);
     // do not load sort index file for reverse dictionary
     checkAndLoadDictionaryData(dictionaryColumnUniqueIdentifier, columnReverseDictionaryInfo,
-        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier(),
+        getLruCacheKey(dictionaryColumnUniqueIdentifier.getColumnIdentifier().getColumnId(),
             CacheType.REVERSE_DICTIONARY), false);
     reverseDictionary = new ReverseDictionary(columnReverseDictionaryInfo);
     return reverseDictionary;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java b/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
new file mode 100644
index 0000000..928f53b
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
@@ -0,0 +1,91 @@
+package org.carbondata.core.carbon;
+
+import java.io.Serializable;
+import java.util.Map;
+
+import org.carbondata.core.carbon.metadata.datatype.DataType;
+
+/**
+ * Column unique identifier
+ */
+public class ColumnIdentifier implements Serializable {
+
+  /**
+   *
+   */
+  private static final long serialVersionUID = 1L;
+
+  /**
+   * column id
+   */
+  private String columnId;
+
+  /**
+   * column properties
+   */
+  private Map<String, String> columnProperties;
+
+  private DataType dataType;
+
+  /**
+   * @param columnId
+   * @param columnProperties
+   */
+  public ColumnIdentifier(String columnId, Map<String, String> columnProperties,
+      DataType dataType) {
+    this.columnId = columnId;
+    this.columnProperties = columnProperties;
+    this.dataType = dataType;
+  }
+
+  /**
+   * @return columnId
+   */
+  public String getColumnId() {
+    return columnId;
+  }
+
+  /**
+   * @param columnProperty
+   * @return
+   */
+  public String getColumnProperty(String columnProperty) {
+    if (null != columnProperties) {
+      return columnProperties.get(columnProperty);
+    }
+    return null;
+  }
+
+  public DataType getDataType() {
+    return this.dataType;
+  }
+
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((columnId == null) ? 0 : columnId.hashCode());
+    return result;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    ColumnIdentifier other = (ColumnIdentifier) obj;
+    if (columnId == null) {
+      if (other.columnId != null) {
+        return false;
+      }
+    } else if (!columnId.equals(other.columnId)) {
+      return false;
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
index 249d2a5..23394ba 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -159,7 +159,9 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     thriftColumnSchema.setPrecision(wrapperColumnSchema.getPrecision());
     thriftColumnSchema.setNum_child(wrapperColumnSchema.getNumberOfChild());
     thriftColumnSchema.setDefault_value(wrapperColumnSchema.getDefaultValue());
+    thriftColumnSchema.setColumnProperties(wrapperColumnSchema.getColumnProperties());
     thriftColumnSchema.setInvisible(wrapperColumnSchema.isInvisible());
+    thriftColumnSchema.setColumnReferenceId(wrapperColumnSchema.getColumnReferenceId());
     return thriftColumnSchema;
   }
 
@@ -324,7 +326,9 @@ public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
     wrapperColumnSchema.setScale(externalColumnSchema.getScale());
     wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
     wrapperColumnSchema.setAggregateFunction(externalColumnSchema.getAggregate_function());
+    wrapperColumnSchema.setColumnProperties(externalColumnSchema.getColumnProperties());
     wrapperColumnSchema.setInvisible(externalColumnSchema.isInvisible());
+    wrapperColumnSchema.setColumnReferenceId(externalColumnSchema.getColumnReferenceId());
     return wrapperColumnSchema;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
index 331c661..04008ab 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
@@ -21,7 +21,9 @@ package org.carbondata.core.carbon.metadata.schema.table.column;
 
 import java.io.Serializable;
 import java.util.List;
+import java.util.Map;
 
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
 
@@ -48,9 +50,16 @@ public class CarbonColumn implements Serializable {
    */
   protected byte[] defaultValue;
 
+  /**
+   * Column identifier
+   */
+  protected ColumnIdentifier columnIdentifier;
+
   public CarbonColumn(ColumnSchema columnSchema, int ordinal) {
     this.columnSchema = columnSchema;
     this.ordinal = ordinal;
+    this.columnIdentifier =
+        new ColumnIdentifier(getColumnId(), getColumnProperties(), getDataType());
   }
 
   /**
@@ -142,4 +151,18 @@ public class CarbonColumn implements Serializable {
   public ColumnSchema getColumnSchema() {
     return this.columnSchema;
   }
+
+  /**
+   * @return columnproperty
+   */
+  public Map<String, String> getColumnProperties() {
+    return this.columnSchema.getColumnProperties();
+  }
+
+  /**
+   * @return columnIdentifier
+   */
+  public ColumnIdentifier getColumnIdentifier() {
+    return this.columnIdentifier;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
index f48c53f..5c54ad3 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
@@ -20,6 +20,7 @@ package org.carbondata.core.carbon.metadata.schema.table.column;
 
 import java.io.Serializable;
 import java.util.List;
+import java.util.Map;
 
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.metadata.encoder.Encoding;
@@ -54,6 +55,11 @@ public class ColumnSchema implements Serializable {
   private String columnUniqueId;
 
   /**
+   * column reference id
+   */
+  private String columnReferenceId;
+
+  /**
    * whether it is stored as columnar format or row format
    */
   private boolean isColumnar = true;
@@ -100,6 +106,11 @@ public class ColumnSchema implements Serializable {
   private byte[] defaultValue;
 
   /**
+   * Column properties
+   */
+  private Map<String, String> columnProperties;
+
+  /**
    * used to define the column visibility of column default is false
    */
   private boolean invisible = false;
@@ -332,6 +343,30 @@ public class ColumnSchema implements Serializable {
   }
 
   /**
+   * @param columnProperties
+   */
+  public void setColumnProperties(Map<String, String> columnProperties) {
+    this.columnProperties = columnProperties;
+  }
+
+  /**
+   * @param property
+   * @return
+   */
+  public String getColumnProperty(String property) {
+    if (null != columnProperties) {
+      return columnProperties.get(property);
+    }
+    return null;
+  }
+
+  /**
+   * return columnproperties
+   */
+  public Map<String, String> getColumnProperties() {
+    return columnProperties;
+  }
+  /**
    * return the visibility
    * @return
    */
@@ -347,4 +382,18 @@ public class ColumnSchema implements Serializable {
     this.invisible = invisible;
   }
 
+  /**
+   * @return columnReferenceId
+   */
+  public String getColumnReferenceId() {
+    return columnReferenceId;
+  }
+
+  /**
+   * @param columnReferenceId
+   */
+  public void setColumnReferenceId(String columnReferenceId) {
+    this.columnReferenceId = columnReferenceId;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
index 5f862bc..538a16b 100644
--- a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
+++ b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
@@ -35,21 +35,21 @@ import org.apache.hadoop.fs.Path;
  */
 public class CarbonTablePath extends Path {
 
-  private static final String METADATA_DIR = "Metadata";
+  protected static final String METADATA_DIR = "Metadata";
   private static final String DICTIONARY_EXT = ".dict";
   private static final String DICTIONARY_META_EXT = ".dictmeta";
   private static final String SORT_INDEX_EXT = ".sortindex";
-  private static final String SCHEMA_FILE = "schema";
-  private static final String TABLE_STATUS_FILE = "tablestatus";
-  private static final String FACT_DIR = "Fact";
-  private static final String AGGREGATE_TABLE_PREFIX = "Agg";
-  private static final String SEGMENT_PREFIX = "Segment_";
-  private static final String PARTITION_PREFIX = "Part";
-  private static final String CARBON_DATA_EXT = ".carbondata";
-  private static final String DATA_PART_PREFIX = "part";
+  protected static final String SCHEMA_FILE = "schema";
+  protected static final String TABLE_STATUS_FILE = "tablestatus";
+  protected static final String FACT_DIR = "Fact";
+  protected static final String AGGREGATE_TABLE_PREFIX = "Agg";
+  protected static final String SEGMENT_PREFIX = "Segment_";
+  protected static final String PARTITION_PREFIX = "Part";
+  protected static final String CARBON_DATA_EXT = ".carbondata";
+  protected static final String DATA_PART_PREFIX = "part";
   private static final String INDEX_FILE_EXT = ".carbonindex";
 
-  private String tablePath;
+  protected String tablePath;
 
   public CarbonTablePath(String tablePathString) {
     super(tablePathString);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 473e764..39b80d1 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -719,6 +719,14 @@ public final class CarbonCommonConstants {
   public static final String CARBON_MERGE_SORT_PREFETCH = "carbon.merge.sort.prefetch";
   public static final String CARBON_MERGE_SORT_PREFETCH_DEFAULT = "true";
 
+  // tblproperties
+  public static final String COLUMN_GROUPS = "column_groups";
+  public static final String DICTIONARY_EXCLUDE = "dictionary_exclude";
+  public static final String DICTIONARY_INCLUDE = "dictionary_include";
+  public static final String PARTITIONCLASS = "partitionclass";
+  public static final String PARTITIONCOUNT = "partitioncount";
+  public static final String COLUMN_PROPERTIES = "columnproperties";
+
   /**
    * this variable is to enable/disable identify high cardinality during first data loading
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
index f76cd49..3b4d271 100644
--- a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
@@ -24,6 +24,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -47,9 +48,9 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
   private String hdfsStorePath;
 
   /**
-   * column name
+   * column identifier
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * dictionary metadata file path
@@ -69,7 +70,7 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionaryMetadataReaderImpl(String hdfsStorePath,
-      CarbonTableIdentifier carbonTableIdentifier, String columnIdentifier) {
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier) {
     this.hdfsStorePath = hdfsStorePath;
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
@@ -155,7 +156,7 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
     this.columnDictionaryMetadataFilePath =
-        carbonTablePath.getDictionaryMetaFilePath(columnIdentifier);
+        carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
index d773f68..addedf6 100644
--- a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
@@ -25,6 +25,7 @@ import java.util.ArrayList;
 import java.util.List;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -51,7 +52,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   /**
    * column name
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * dictionary file path
@@ -71,7 +72,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionaryReaderImpl(String hdfsStorePath,
-      CarbonTableIdentifier carbonTableIdentifier, String columnIdentifier) {
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier) {
     this.hdfsStorePath = hdfsStorePath;
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
@@ -199,7 +200,8 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   private void initFileLocation() {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
-    this.columnDictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier);
+    this.columnDictionaryFilePath = carbonTablePath
+        .getDictionaryFilePath(columnIdentifier.getColumnId());
   }
 
   /**
@@ -266,9 +268,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
    * @throws IOException read and close method throws IO exception
    */
   private List<CarbonDictionaryColumnMetaChunk> readDictionaryMetadataFile() throws IOException {
-    CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.hdfsStorePath, this.carbonTableIdentifier,
-            this.columnIdentifier);
+    CarbonDictionaryMetadataReader columnMetadataReaderImpl = getDictionaryMetadataReader();
     List<CarbonDictionaryColumnMetaChunk> dictionaryMetaChunkList = null;
     // read metadata file
     try {
@@ -281,6 +281,14 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   }
 
   /**
+   * @return
+   */
+  protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
+    return new CarbonDictionaryMetadataReaderImpl(this.hdfsStorePath, carbonTableIdentifier,
+        this.columnIdentifier);
+  }
+
+  /**
    * This method will open the dictionary file stream for reading
    *
    * @throws IOException thrift reader open method throws IOException

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
index b84ae54..eb3f8bd 100644
--- a/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.reader.ThriftReader;
@@ -44,7 +45,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   /**
    * column name
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * hdfs store location
@@ -78,7 +79,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
    * @param carbonStorePath       carbon store path
    */
   public CarbonDictionarySortIndexReaderImpl(final CarbonTableIdentifier carbonTableIdentifier,
-      final String columnIdentifier, final String carbonStorePath) {
+      final ColumnIdentifier columnIdentifier, final String carbonStorePath) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.carbonStorePath = carbonStorePath;
@@ -144,7 +145,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   private void init() throws IOException {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
-    this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier);
+    this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier.getColumnId());
     openThriftReader();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java b/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
new file mode 100644
index 0000000..9c65e3f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.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.core.service;
+
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Column Unique id generator
+ */
+public interface ColumnUniqueIdService {
+
+  /**
+   * @param schemaName
+   * @param columnSchema
+   * @return generate unique id
+   */
+  public String generateUniqueId(String schemaName, ColumnSchema columnSchema);
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/service/DictionaryService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/service/DictionaryService.java b/core/src/main/java/org/carbondata/core/service/DictionaryService.java
new file mode 100644
index 0000000..97afe29
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/service/DictionaryService.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.core.service;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
+import org.carbondata.core.reader.CarbonDictionaryReader;
+import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
+import org.carbondata.core.writer.CarbonDictionaryWriter;
+import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
+
+/**
+ * Dictionary service to get writer and reader
+ */
+public interface DictionaryService {
+
+  /**
+   * get dictionary writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionaryWriter getDictionaryWriter(CarbonTableIdentifier carbonTableIdentifier,
+      ColumnIdentifier columnIdentifier, String carbonStorePath);
+
+  /**
+   * get dictionary sort index writer
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionarySortIndexWriter getDictionarySortIndexWriter(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath);
+
+  /**
+   * get dictionary metadata reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionaryMetadataReader getDictionaryMetadataReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath);
+
+  /**
+   * get dictionary reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionaryReader getDictionaryReader(CarbonTableIdentifier carbonTableIdentifier,
+      ColumnIdentifier columnIdentifier, String carbonStorePath);
+
+  /**
+   * get dictionary sort index reader
+   *
+   * @param carbonTableIdentifier
+   * @param columnIdentifier
+   * @param carbonStorePath
+   * @return
+   */
+  public CarbonDictionarySortIndexReader getDictionarySortIndexReader(
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier,
+      String carbonStorePath);
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/service/PathService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/service/PathService.java b/core/src/main/java/org/carbondata/core/service/PathService.java
new file mode 100644
index 0000000..7ef3a45
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/service/PathService.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.core.service;
+
+import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
+import org.carbondata.core.carbon.path.CarbonTablePath;
+
+/**
+ * Create helper to get path details
+ */
+public interface PathService {
+
+  /**
+   * @param columnIdentifier
+   * @param storeLocation
+   * @param tableIdentifier
+   * @return store path related to tables
+   */
+  CarbonTablePath getCarbonTablePath(ColumnIdentifier columnIdentifier, String storeLocation,
+      CarbonTableIdentifier tableIdentifier);
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 0f94c5a..886e136 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -1657,6 +1657,70 @@ public final class CarbonUtil {
   }
 
   /**
+   * convert from wrapper to external data type
+   *
+   * @param dataType
+   * @return
+   */
+  public static org.carbondata.format.DataType fromWrapperToExternalDataType(DataType dataType) {
+
+    if (null == dataType) {
+      return null;
+    }
+    switch (dataType) {
+      case STRING:
+        return org.carbondata.format.DataType.STRING;
+      case INT:
+        return org.carbondata.format.DataType.INT;
+      case LONG:
+        return org.carbondata.format.DataType.LONG;
+      case DOUBLE:
+        return org.carbondata.format.DataType.DOUBLE;
+      case DECIMAL:
+        return org.carbondata.format.DataType.DECIMAL;
+      case TIMESTAMP:
+        return org.carbondata.format.DataType.TIMESTAMP;
+      case ARRAY:
+        return org.carbondata.format.DataType.ARRAY;
+      case STRUCT:
+        return org.carbondata.format.DataType.STRUCT;
+      default:
+        return org.carbondata.format.DataType.STRING;
+    }
+  }
+
+  /**
+   * convert from external to wrapper data type
+   *
+   * @param dataType
+   * @return
+   */
+  public static DataType fromExternalToWrapperDataType(org.carbondata.format.DataType dataType) {
+    if (null == dataType) {
+      return null;
+    }
+    switch (dataType) {
+      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;
+    }
+  }
+  /**
    * @param dictionaryColumnCardinality
    * @param wrapperColumnSchemaList
    * @return It returns formatted cardinality by adding -1 value for NoDictionary columns

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index c6bc343..22e5bbe 100644
--- a/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -25,16 +25,19 @@ import java.nio.charset.Charset;
 import java.util.ArrayList;
 import java.util.List;
 
+import org.carbondata.common.factory.CarbonCommonFactory;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
-import org.carbondata.core.carbon.path.CarbonStorePath;
+import org.carbondata.core.carbon.ColumnIdentifier;
 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.reader.CarbonDictionaryColumnMetaChunk;
+import org.carbondata.core.reader.CarbonDictionaryMetadataReader;
 import org.carbondata.core.reader.CarbonDictionaryMetadataReaderImpl;
+import org.carbondata.core.service.PathService;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.format.ColumnDictionaryChunk;
@@ -56,7 +59,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   /**
    * carbon type identifier
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  protected CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * list which will hold values upto maximum of one dictionary chunk size
@@ -74,14 +77,14 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   private ThriftWriter dictionaryThriftWriter;
 
   /**
-   * column name
+   * column identifier
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * HDFS store path
    */
-  private String hdfsStorePath;
+  protected String hdfsStorePath;
 
   /**
    * dictionary file path
@@ -134,7 +137,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionaryWriterImpl(String hdfsStorePath,
-      CarbonTableIdentifier carbonTableIdentifier, String columnIdentifier) {
+      CarbonTableIdentifier carbonTableIdentifier, ColumnIdentifier columnIdentifier) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.hdfsStorePath = hdfsStorePath;
@@ -236,10 +239,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
    */
   private void init() throws IOException {
     initDictionaryChunkSize();
-    CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
-    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier);
-    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier);
+    initPaths();
     if (CarbonUtil.isFileExists(this.dictionaryFilePath)) {
       this.chunk_start_offset = CarbonUtil.getFileSize(this.dictionaryFilePath);
       validateDictionaryFileOffsetWithLastSegmentEntryOffset();
@@ -248,6 +248,15 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
     createChunkList();
   }
 
+  protected void initPaths() {
+    PathService pathService = CarbonCommonFactory.getPathService();
+    CarbonTablePath carbonTablePath = pathService.getCarbonTablePath(columnIdentifier,
+            this.hdfsStorePath, carbonTableIdentifier);
+    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier.getColumnId());
+    this.dictionaryMetaFilePath =
+        carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
+  }
+
   /**
    * initialize the value of dictionary chunk that can be kept in memory at a time
    */
@@ -382,9 +391,7 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   private CarbonDictionaryColumnMetaChunk getChunkMetaObjectForLastSegmentEntry()
       throws IOException {
     CarbonDictionaryColumnMetaChunk carbonDictionaryColumnMetaChunk = null;
-    CarbonDictionaryMetadataReaderImpl columnMetadataReaderImpl =
-        new CarbonDictionaryMetadataReaderImpl(this.hdfsStorePath, this.carbonTableIdentifier,
-            this.columnIdentifier);
+    CarbonDictionaryMetadataReader columnMetadataReaderImpl = getDictionaryMetadataReader();
     try {
       // read the last segment entry for dictionary metadata
       carbonDictionaryColumnMetaChunk =
@@ -395,4 +402,12 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
     }
     return carbonDictionaryColumnMetaChunk;
   }
+
+  /**
+   * @return
+   */
+  protected CarbonDictionaryMetadataReader getDictionaryMetadataReader() {
+    return new CarbonDictionaryMetadataReaderImpl(hdfsStorePath, carbonTableIdentifier,
+        columnIdentifier);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java b/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
index 5ae87cd..d28b176 100644
--- a/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
+++ b/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
@@ -24,6 +24,7 @@ import java.util.List;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.util.CarbonUtil;
@@ -39,17 +40,17 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
   /**
    * carbonTable Identifier holding the info of databaseName and tableName
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  protected CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * column name
    */
-  private String columnIdentifier;
+  protected ColumnIdentifier columnIdentifier;
 
   /**
    * carbon store location
    */
-  private String carbonStorePath;
+  protected String carbonStorePath;
   /**
    * Path of dictionary sort index file for which the sortIndex to be written
    */
@@ -76,7 +77,7 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
    * @param columnIdentifier      column unique identifier
    */
   public CarbonDictionarySortIndexWriterImpl(final CarbonTableIdentifier carbonTableIdentifier,
-      final String columnIdentifier, final String carbonStorePath) {
+      final ColumnIdentifier columnIdentifier, final String carbonStorePath) {
     this.carbonTableIdentifier = carbonTableIdentifier;
     this.columnIdentifier = columnIdentifier;
     this.carbonStorePath = carbonStorePath;
@@ -117,9 +118,7 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
     boolean isNotNull =
         null != columnSortInfo.getSort_index() && null != columnSortInfo.sort_index_inverted;
     if (isNotNull) {
-      CarbonTablePath carbonTablePath =
-          CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
-      this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier);
+      initPath();
       String folderContainingFile = CarbonTablePath.getFolderContainingFile(this.sortIndexFilePath);
       boolean created = CarbonUtil.checkAndCreateFolder(folderContainingFile);
       if (!created) {
@@ -144,6 +143,12 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
     }
   }
 
+  protected void initPath() {
+    CarbonTablePath carbonTablePath =
+        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
+    this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier.getColumnId());
+  }
+
   /**
    * Closes this stream and releases any system resources associated
    * with it. If the stream is already closed then invoking this

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 1d68872..84cb975 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
@@ -357,16 +357,16 @@ public class QueryUtil {
         CarbonMetadata.getInstance().getCarbonTable(carbonTableIdentifier.getTableUniqueName());
     List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
         new ArrayList<>(dictionaryColumnIdList.size());
-    for (String columnIdentifier : dictionaryColumnIdList) {
+    for (String columnId : dictionaryColumnIdList) {
       CarbonDimension dimension = CarbonMetadata.getInstance()
-          .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnIdentifier);
+          .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnId);
       if (null == dimension) {
         throw new QueryExecutionException(
-            "The column id " + columnIdentifier + " could not be resolved.");
+            "The column id " + columnId + " could not be resolved.");
       }
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
-          new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
-              dimension.getDataType());
+          new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
+              dimension.getColumnIdentifier(), dimension.getDataType());
       dictionaryColumnUniqueIdentifiers.add(dictionaryColumnUniqueIdentifier);
     }
     return dictionaryColumnUniqueIdentifiers;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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
index 74cb4cd..810cbaf 100644
--- 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
@@ -871,7 +871,7 @@ public final class FilterUtil {
       CarbonDimension carbonDimension) throws QueryExecutionException {
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(tableIdentifier.getCarbonTableIdentifier(),
-            String.valueOf(carbonDimension.getColumnId()), carbonDimension.getDataType());
+            carbonDimension.getColumnIdentifier(), carbonDimension.getDataType());
     CacheProvider cacheProvider = CacheProvider.getInstance();
     Cache forwardDictionaryCache =
         cacheProvider.createCache(CacheType.FORWARD_DICTIONARY, tableIdentifier.getStorePath());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java b/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
index be9566f..3a23d0d 100644
--- a/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
+++ b/core/src/test/java/org/carbondata/core/cache/dictionary/AbstractDictionaryCacheTest.java
@@ -14,6 +14,7 @@ import java.util.Properties;
 
 import org.carbondata.core.cache.Cache;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.datatype.DataType;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
@@ -84,7 +85,8 @@ public class AbstractDictionaryCacheTest {
   }
 
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
-      String columnIdentifier) {
+      String columnId) {
+	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, DataType.STRING);
     DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
         new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
             DataType.STRING);
@@ -106,8 +108,9 @@ public class AbstractDictionaryCacheTest {
    * @param data
    * @throws IOException
    */
-  protected void prepareWriterAndWriteData(List<String> data, String columnIdentifier)
+  protected void prepareWriterAndWriteData(List<String> data, String columnId)
       throws IOException {
+	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     CarbonDictionaryWriter carbonDictionaryWriter =
         new CarbonDictionaryWriterImpl(carbonStorePath, carbonTableIdentifier, columnIdentifier);
     CarbonTablePath carbonTablePath =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java b/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
index fab0e97..ad6787b 100644
--- a/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
+++ b/core/src/test/java/org/carbondata/core/cache/dictionary/ForwardDictionaryCacheTest.java
@@ -31,6 +31,7 @@ import org.carbondata.core.cache.Cache;
 import org.carbondata.core.cache.CacheProvider;
 import org.carbondata.core.cache.CacheType;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
@@ -208,7 +209,8 @@ public class ForwardDictionaryCacheTest extends AbstractDictionaryCacheTest {
    * @param columnIdentifier
    * @throws IOException
    */
-  private void writeSortIndexFile(List<String> data, String columnIdentifier) throws IOException {
+  private void writeSortIndexFile(List<String> data, String columnId) throws IOException {
+	ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
     Map<String, Integer> dataToSurrogateKeyMap = new HashMap<>(data.size());
     int surrogateKey = 0;
     List<Integer> invertedIndexList = new ArrayList<>(data.size());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java b/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
index ef8f43d..187e5e4 100644
--- a/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
+++ b/core/src/test/java/org/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
@@ -32,6 +32,7 @@ import org.carbondata.core.cache.Cache;
 import org.carbondata.core.cache.CacheProvider;
 import org.carbondata.core.cache.CacheType;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.reader.CarbonDictionaryColumnMetaChunk;
 import org.carbondata.core.util.CarbonProperties;
@@ -264,7 +265,8 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
     }
   }
   protected DictionaryColumnUniqueIdentifier createDictionaryColumnUniqueIdentifier(
-	      String columnIdentifier) {
+	      String columnId) {
+	    ColumnIdentifier columnIdentifier = new ColumnIdentifier(columnId, null, null);
 	    DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
 	        new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier);
 	    return dictionaryColumnUniqueIdentifier;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java b/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
index 64767e1..8b10c32 100644
--- a/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
+++ b/core/src/test/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImplTest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.writer.sortindex.CarbonDictionarySortIndexWriter;
@@ -58,8 +59,9 @@ public class CarbonDictionarySortIndexReaderImplTest {
     deleteStorePath();
     CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("testSchema", "carbon",
     		UUID.randomUUID().toString());
+    ColumnIdentifier columnIdentifier = new ColumnIdentifier("Name", null, null);
     CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, "Name", hdfsStorePath);
+        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, columnIdentifier, hdfsStorePath);
     List<int[]> expectedData = prepareExpectedData();
 
     List<Integer> sortIndex = Arrays.asList(ArrayUtils.toObject(expectedData.get(0)));
@@ -68,7 +70,7 @@ public class CarbonDictionarySortIndexReaderImplTest {
     dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
     dictionarySortIndexWriter.close();
     CarbonDictionarySortIndexReader dictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, "Name", hdfsStorePath);
+        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier, hdfsStorePath);
     List<Integer> actualSortIndex = dictionarySortIndexReader.readSortIndex();
     List<Integer> actualInvertedSortIndex = dictionarySortIndexReader.readInvertedSortIndex();
     for (int i = 0; i < actualSortIndex.size(); i++) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java b/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
index 6c03636..31822d1 100644
--- a/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
+++ b/core/src/test/java/org/carbondata/core/writer/CarbonDictionaryWriterImplTest.java
@@ -33,6 +33,7 @@ import java.util.Properties;
 import java.util.UUID;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.path.CarbonStorePath;
 import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
@@ -71,7 +72,7 @@ public class CarbonDictionaryWriterImplTest {
 
   private String carbonStorePath;
 
-  private String columnIdentifier;
+  private ColumnIdentifier columnIdentifier;
 
   private Properties props;
 
@@ -96,7 +97,7 @@ public class CarbonDictionaryWriterImplTest {
     this.databaseName = props.getProperty("database", "testSchema");
     this.tableName = props.getProperty("tableName", "carbon");
     this.carbonStorePath = props.getProperty("storePath", "carbonStore");
-    this.columnIdentifier = "Name";
+    this.columnIdentifier = new ColumnIdentifier("Name", null, null);
     carbonTableIdentifier = new CarbonTableIdentifier(databaseName, tableName, UUID.randomUUID().toString());
     deleteStorePath();
     prepareDataSet();
@@ -514,7 +515,7 @@ public class CarbonDictionaryWriterImplTest {
     if(!FileFactory.isFileExist(dictionaryLocation, fileType)) {
       FileFactory.mkdirs(dictionaryLocation, fileType);
     }
-    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier);
-    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier);
+    this.dictionaryFilePath = carbonTablePath.getDictionaryFilePath(columnIdentifier.getColumnId());
+    this.dictionaryMetaFilePath = carbonTablePath.getDictionaryMetaFilePath(columnIdentifier.getColumnId());
   }
 }


[15/50] [abbrv] incubator-carbondata git commit: [issue-CARBONDATA-12] Carbon data load bad record is not written into the bad record log file (#753)

Posted by ch...@apache.org.
[issue-CARBONDATA-12] Carbon data load bad record is not written into the bad record log file (#753)



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

Branch: refs/heads/master
Commit: 148285df54b799e866033df3fac7541ffbea17f6
Parents: 720e8d6
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Sat Jun 25 20:20:58 2016 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sat Jun 25 20:20:58 2016 +0530

----------------------------------------------------------------------
 .../carbondata/processing/merger/step/CarbonSliceMergerStep.java  | 3 ++-
 1 file changed, 2 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/148285df/processing/src/main/java/org/carbondata/processing/merger/step/CarbonSliceMergerStep.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/merger/step/CarbonSliceMergerStep.java b/processing/src/main/java/org/carbondata/processing/merger/step/CarbonSliceMergerStep.java
index c075999..b41b88f 100644
--- a/processing/src/main/java/org/carbondata/processing/merger/step/CarbonSliceMergerStep.java
+++ b/processing/src/main/java/org/carbondata/processing/merger/step/CarbonSliceMergerStep.java
@@ -128,7 +128,8 @@ public class CarbonSliceMergerStep extends BaseStep {
 
   private void renameFolders() {
     CarbonDataProcessorUtil.renameBadRecordsFromInProgressToNormal(
-        meta.getSchemaName() + File.separator + meta.getCubeName());
+        meta.getSchemaName() + File.separator + meta.getCubeName() + File.separator + meta
+            .getTaskNo());
   }
 
   /**


[13/50] [abbrv] incubator-carbondata git commit: [BUG] Memory leak issue , Clean Up handling, Preserve handling during compaction (#748)

Posted by ch...@apache.org.
[BUG] Memory leak issue ,Clean Up handling,Preserve handling during compaction (#748)

* handling the cleanup of stale compacted segments in the compaction also.
* Fix problem in preserving segments during compaction
* Memory leak problem in query flow.

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

Branch: refs/heads/master
Commit: 24c47c204426cdc51d6d3d13a47f4d351b687484
Parents: 322a77b
Author: ravikiran23 <ra...@gmail.com>
Authored: Sat Jun 25 18:56:14 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Sat Jun 25 18:56:14 2016 +0530

----------------------------------------------------------------------
 .../impl/InternalDetailQueryExecutor.java       |  6 +++
 .../result/iterator/RawResultIterator.java      |  2 +
 .../carbondata/spark/load/CarbonLoaderUtil.java | 26 ++++++++----
 .../spark/merger/CarbonDataMergerUtil.java      | 37 ++++-------------
 .../spark/rdd/CarbonDataRDDFactory.scala        | 43 ++++++++++++++++++--
 5 files changed, 74 insertions(+), 40 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/24c47c20/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..72a7f62 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
@@ -123,6 +123,12 @@ public class InternalDetailQueryExecutor implements InternalQueryExecutor {
       throw new QueryExecutionException(e);
     } finally {
       execService = null;
+      for (int currentSliceIndex : sliceIndexes) {
+        if (currentSliceIndex == -1) {
+          continue;
+        }
+        executionInfos.get(currentSliceIndex).setScannedResultProcessor(null);
+      }
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/24c47c20/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
index 27114f2..8977799 100644
--- 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
@@ -70,6 +70,7 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
 
     if (null == batch || checkIfBatchIsProcessedCompletely(batch)) {
       if (detailRawQueryResultIterator.hasNext()) {
+        batch = null;
         batch = detailRawQueryResultIterator.next();
         counter = 0; // batch changed so reset the counter.
       } else {
@@ -102,6 +103,7 @@ public class RawResultIterator extends CarbonIterator<Object[]> {
         return null;
       }
     } else { // completed one batch.
+      batch = null;
       batch = detailRawQueryResultIterator.next();
       counter = 0;
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/24c47c20/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 427c7e5..0fb2621 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
@@ -285,18 +285,19 @@ public final class CarbonLoaderUtil {
     }
   }
 
-  public static void deletePartialLoadDataIfExist(CarbonLoadModel loadModel) throws IOException {
+  public static void deletePartialLoadDataIfExist(CarbonLoadModel loadModel,
+      final boolean isCompactionFlow) throws IOException {
     CarbonTable carbonTable = loadModel.getCarbonDataLoadSchema().getCarbonTable();
     String metaDataLocation = carbonTable.getMetaDataFilepath();
     SegmentStatusManager segmentStatusManager =
         new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier());
     LoadMetadataDetails[] details = segmentStatusManager.readLoadMetadata(metaDataLocation);
-    CarbonTablePath carbonTablePath = CarbonStorePath.getCarbonTablePath(loadModel.getStorePath(),
-        carbonTable.getCarbonTableIdentifier());
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(loadModel.getStorePath(), carbonTable.getCarbonTableIdentifier());
     final List<String> loadFolders = new ArrayList<String>();
     for (LoadMetadataDetails loadMetadata : details) {
-      loadFolders.add(carbonTablePath.getCarbonDataDirectoryPath(loadMetadata.getPartitionCount(),
-          loadMetadata.getLoadName())
+      loadFolders.add(carbonTablePath
+          .getCarbonDataDirectoryPath(loadMetadata.getPartitionCount(), loadMetadata.getLoadName())
           .replace("\\", "/"));
     }
 
@@ -307,13 +308,22 @@ public final class CarbonLoaderUtil {
       if (FileFactory.isFileExist(partitionPath, fileType)) {
         CarbonFile carbonFile = FileFactory.getCarbonFile(partitionPath, fileType);
         CarbonFile[] listFiles = carbonFile.listFiles(new CarbonFileFilter() {
-          @Override
-          public boolean accept(CarbonFile path) {
+          @Override public boolean accept(CarbonFile path) {
             return !loadFolders.contains(path.getAbsolutePath().replace("\\", "/"));
           }
         });
         for (int k = 0; k < listFiles.length; k++) {
-          deleteStorePath(listFiles[k].getAbsolutePath());
+          String segmentId =
+              CarbonTablePath.DataPathUtil.getSegmentId(listFiles[k].getAbsolutePath() + "/dummy");
+          if (isCompactionFlow) {
+            if (segmentId.contains(".")) {
+              deleteStorePath(listFiles[k].getAbsolutePath());
+            }
+          } else {
+            if (!segmentId.contains(".")) {
+              deleteStorePath(listFiles[k].getAbsolutePath());
+            }
+          }
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/24c47c20/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
index 7accb91..52e8a08 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
@@ -588,39 +588,20 @@ public final class CarbonDataMergerUtil {
       }
     }
 
-    // handle the retaining of valid loads,
-
     // check if valid list is big enough for removing the number of seg to be retained.
-    if (validList.size() > numberOfSegToBeRetained) {
-
-      // after the sort remove the loads from the last as per the retaining count.
-      Collections.sort(validList, new Comparator<LoadMetadataDetails>() {
-
-        @Override public int compare(LoadMetadataDetails seg1, LoadMetadataDetails seg2) {
-          double segNumber1 = Double.parseDouble(seg1.getLoadName());
-          double segNumber2 = Double.parseDouble(seg2.getLoadName());
-
-          if ((segNumber1 - segNumber2) < 0) {
-            return -1;
-          } else if ((segNumber1 - segNumber2) > 0) {
-            return 1;
-          }
-          return 0;
-
-        }
-      });
-
-      for (int i = 0; i < numberOfSegToBeRetained; i++) {
-
-        // remove last segment
-        validList.remove(validList.size() - 1);
+    // last element
+    int removingIndex = validList.size() - 1;
 
+    for (int i = validList.size(); i > 0; i--) {
+      if (numberOfSegToBeRetained == 0) {
+        break;
       }
-      return validList;
+      // remove last segment
+      validList.remove(removingIndex--);
+      numberOfSegToBeRetained--;
     }
+    return validList;
 
-    // case where there is no 2 loads available for merging.
-    return new ArrayList<LoadMetadataDetails>(0);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/24c47c20/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 e534e3d..6f44aef 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
@@ -369,6 +369,18 @@ object CarbonDataRDDFactory extends Logging {
     readLoadMetadataDetails(carbonLoadModel, hdfsStoreLocation)
     var segList: util.List[LoadMetadataDetails] = carbonLoadModel.getLoadMetadataDetails
 
+    // clean up of the stale segments.
+    try {
+      CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, true)
+    }
+    catch {
+      case e: Exception =>
+        logger
+          .error("Exception in compaction thread while clean up of stale segments " + e
+            .getMessage
+          )
+    }
+
     var loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
       hdfsStoreLocation,
       carbonLoadModel,
@@ -384,11 +396,25 @@ object CarbonDataRDDFactory extends Logging {
         override def run(): Unit = {
 
           while (loadsToMerge.size() > 1) {
-
+          // Deleting the any partially loaded data if present.
+          // in some case the segment folder which is present in store will not have entry in
+          // status.
+          // so deleting those folders.
+          try {
+            CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, true)
+          }
+          catch {
+            case e: Exception =>
+              logger
+                .error("Exception in compaction thread while clean up of stale segments " + e
+                  .getMessage
+                )
+          }
             val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
               CarbonCommonConstants
                 .DEFAULT_COLLECTION_SIZE
             )
+
             scanSegmentsAndSubmitJob(futureList)
 
             futureList.asScala.foreach(future => {
@@ -425,7 +451,8 @@ object CarbonDataRDDFactory extends Logging {
 
     /**
      * This will scan all the segments and submit the loads to be merged into the executor.
-     * @param futureList
+      *
+      * @param futureList
      */
     def scanSegmentsAndSubmitJob(futureList: util.List[Future[Void]]): Unit = {
       breakable {
@@ -579,8 +606,16 @@ object CarbonDataRDDFactory extends Logging {
       // Deleting the any partially loaded data if present.
       // in some case the segment folder which is present in store will not have entry in status.
       // so deleting those folders.
-      CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel)
-
+      try {
+        CarbonLoaderUtil.deletePartialLoadDataIfExist(carbonLoadModel, false)
+      }
+      catch {
+        case e: Exception =>
+          logger
+            .error("Exception in data load while clean up of stale segments " + e
+              .getMessage
+            )
+      }
 
       // reading the start time of data load.
       val loadStartTime = CarbonLoaderUtil.readCurrentTime()


[49/50] [abbrv] incubator-carbondata git commit: Resolving compiling issues after merge

Posted by ch...@apache.org.
Resolving compiling 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/4444c324
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/4444c324
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/4444c324

Branch: refs/heads/master
Commit: 4444c324183d91a5b0e44617cd3e5d5d60a00fc0
Parents: 7f72218
Author: ravipesala <ra...@gmail.com>
Authored: Thu Jun 30 19:59:51 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Jun 30 19:59:51 2016 +0530

----------------------------------------------------------------------
 .../spark/merger/RowResultMerger.java           |  1 -
 .../carbondata/spark/load/CarbonLoaderUtil.java | 27 +++++++++----
 .../spark/merger/CarbonDataMergerUtil.java      |  2 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     | 10 +++--
 .../org/apache/spark/sql/CarbonSqlParser.scala  | 12 +++---
 .../execution/command/carbonTableSchema.scala   | 19 ++++-----
 .../spark/sql/hive/CarbonStrategies.scala       | 41 +++++++++++++++++++-
 .../datacompaction/DataCompactionLockTest.scala | 10 ++---
 .../ColumnGroupDataTypesTestCase.scala          | 12 +++---
 .../processing/mdkeygen/MDKeyGenStep.java       |  1 -
 .../store/writer/AbstractFactDataWriter.java    |  4 +-
 11 files changed, 93 insertions(+), 46 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 617efd2..54e32a0 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
@@ -39,7 +39,6 @@ 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.columnar.ColumnGroupModel;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 220d1b7..af880f5 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
@@ -18,9 +18,23 @@
  */
 package org.carbondata.spark.load;
 
-import java.io.*;
+import java.io.BufferedWriter;
+import java.io.DataInputStream;
+import java.io.DataOutputStream;
+import java.io.File;
+import java.io.IOException;
+import java.io.OutputStreamWriter;
 import java.text.SimpleDateFormat;
-import java.util.*;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Date;
+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.Set;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
@@ -42,18 +56,18 @@ 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;
 import org.carbondata.core.datastorage.store.impl.FileFactory.FileType;
 import org.carbondata.core.load.LoadMetadataDetails;
-import org.carbondata.core.locks.ICarbonLock;
 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.lcm.locks.ICarbonLock;
 import org.carbondata.lcm.status.SegmentStatusManager;
 import org.carbondata.processing.api.dataloader.DataLoadModel;
 import org.carbondata.processing.api.dataloader.SchemaInfo;
@@ -67,7 +81,6 @@ import org.carbondata.spark.merger.NodeBlockRelation;
 import org.carbondata.spark.merger.NodeMultiBlockRelation;
 
 import com.google.gson.Gson;
-
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.io.IOUtils;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
index 3aa66c2..f71d7b4 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
@@ -44,9 +44,9 @@ 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.load.LoadMetadataDetails;
-import org.carbondata.core.locks.ICarbonLock;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.integration.spark.merger.CompactionType;
+import org.carbondata.lcm.locks.ICarbonLock;
 import org.carbondata.lcm.status.SegmentStatusManager;
 import org.carbondata.spark.load.CarbonLoadModel;
 import org.carbondata.spark.load.CarbonLoaderUtil;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 8a9f1c9..ab8b297 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
@@ -198,9 +198,13 @@ case class CarbonDictionaryDecoder(
       cache: Cache[DictionaryColumnUniqueIdentifier, Dictionary]) = {
     val dicts: Seq[Dictionary] = getDictionaryColumnIds.map { f =>
       if (f._2 != null) {
-        cache.get(new DictionaryColumnUniqueIdentifier(
-          atiMap.get(f._1).get.getCarbonTableIdentifier,
-          f._2, f._3))
+        try {
+          cache.get(new DictionaryColumnUniqueIdentifier(
+            atiMap.get(f._1).get.getCarbonTableIdentifier,
+            f._2, f._3))
+        } catch {
+          case _ => null
+        }
       } else {
         null
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 406b025..8536544 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
@@ -17,8 +17,6 @@
 
 package org.apache.spark.sql
 
-import java.nio.charset.Charset
-import java.util
 import java.util.regex.{Matcher, Pattern}
 
 import scala.collection.JavaConverters._
@@ -33,7 +31,7 @@ import org.apache.spark.sql.catalyst.{SqlLexical, _}
 import org.apache.spark.sql.catalyst.analysis._
 import org.apache.spark.sql.catalyst.plans.logical._
 import org.apache.spark.sql.catalyst.trees.CurrentOrigin
-import org.apache.spark.sql.execution.command.{DimensionRelation, _}
+import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.datasources.DescribeCommand
 import org.apache.spark.sql.hive.HiveQlWrapper
 
@@ -601,8 +599,8 @@ class CarbonSqlParser()
   }
 
   protected def extractColumnProperties(fields: Seq[Field], tableProperties: Map[String, String]):
-  util.Map[String, util.List[ColumnProperty]] = {
-    val colPropMap = new util.HashMap[String, util.List[ColumnProperty]]()
+  java.util.Map[String, java.util.List[ColumnProperty]] = {
+    val colPropMap = new java.util.HashMap[String, java.util.List[ColumnProperty]]()
     fields.foreach { field =>
       if (field.children.isDefined && field.children.get != null) {
         fillAllChildrenColumnProperty(field.column, field.children, tableProperties, colPropMap)
@@ -615,7 +613,7 @@ class CarbonSqlParser()
 
   protected def fillAllChildrenColumnProperty(parent: String, fieldChildren: Option[List[Field]],
     tableProperties: Map[String, String],
-    colPropMap: util.HashMap[String, util.List[ColumnProperty]]) {
+    colPropMap: java.util.HashMap[String, java.util.List[ColumnProperty]]) {
     fieldChildren.foreach(fields => {
       fields.foreach(field => {
         fillColumnProperty(Some(parent), field.column, tableProperties, colPropMap)
@@ -628,7 +626,7 @@ class CarbonSqlParser()
   protected def fillColumnProperty(parentColumnName: Option[String],
     columnName: String,
     tableProperties: Map[String, String],
-    colPropMap: util.HashMap[String, util.List[ColumnProperty]]) {
+    colPropMap: java.util.HashMap[String, java.util.List[ColumnProperty]]) {
     val (tblPropKey, colProKey) = getKey(parentColumnName, columnName)
     val colProps = CommonUtil.getColumnProperties(tblPropKey, tableProperties)
     if (None != colProps) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 b8afcdf..4a6551b 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
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command
 import java.io.File
 import java.text.SimpleDateFormat
 import java.util
-import java.util.{Date, UUID}
+import java.util.UUID
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
@@ -30,10 +30,8 @@ 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.analysis.TypeCheckResult
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
-import org.apache.spark.sql.catalyst.util.DateTimeUtils
-import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp
+import org.apache.spark.sql.catalyst.TableIdentifier._
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal}
 import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.types.TimestampType
@@ -48,20 +46,19 @@ import org.carbondata.core.carbon.metadata.datatype.DataType
 import org.carbondata.core.carbon.metadata.encoder.Encoding
 import org.carbondata.core.carbon.metadata.schema.{SchemaEvolution, SchemaEvolutionEntry}
 import org.carbondata.core.carbon.metadata.schema.table.{CarbonTable, TableInfo, TableSchema}
-import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
-import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema
+import org.carbondata.core.carbon.metadata.schema.table.column.{CarbonDimension, ColumnSchema}
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.datastorage.store.impl.FileFactory
 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.CarbonSparkFactory
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 import org.carbondata.spark.load._
 import org.carbondata.spark.partition.api.impl.QueryPartitionHelper
 import org.carbondata.spark.rdd.CarbonDataRDDFactory
-import org.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, GlobalDictionaryUtil}
-import org.carbondata.spark.CarbonSparkFactory
+import org.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
 
 
 case class tableModel(
@@ -1568,7 +1565,7 @@ private[sql] case class LoadTable(
       catch {
         case ex: Exception =>
           LOGGER.error(ex)
-          LOGGER.audit(s"Dataload failure for $schemaName.$tableName. Please check the logs")
+          LOGGER.audit(s"Dataload failure for $dbName.$tableName. Please check the logs")
           throw ex
       }
       finally {
@@ -1583,7 +1580,7 @@ private[sql] case class LoadTable(
         } catch {
           case ex: Exception =>
             LOGGER.error(ex)
-            LOGGER.audit(s"Dataload failure for $schemaName.$tableName. " +
+            LOGGER.audit(s"Dataload failure for $dbName.$tableName. " +
               "Problem deleting the partition folder")
             throw ex
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 1821475..1de8908 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
@@ -17,6 +17,8 @@
 
 package org.apache.spark.sql.hive
 
+import java.util
+
 import scala.collection.JavaConverters._
 
 import org.apache.spark.sql._
@@ -32,6 +34,7 @@ import org.apache.spark.sql.execution.command._
 import org.apache.spark.sql.execution.datasources.{DescribeCommand => LogicalDescribeCommand, LogicalRelation}
 import org.apache.spark.sql.hive.execution.{DescribeHiveTableCommand, DropTable, HiveNativeCommand}
 import org.apache.spark.sql.optimizer.{CarbonAliasDecoderRelation, CarbonDecoderRelation}
+import org.apache.spark.sql.types.IntegerType
 
 import org.carbondata.common.logging.LogServiceFactory
 import org.carbondata.spark.exception.MalformedCarbonCommandException
@@ -138,7 +141,11 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         predicates,
         useUnsafeCoversion = false)(sqlContext)
       projectExprsNeedToDecode.addAll(scan.attributesNeedToDecode)
-      if (projectExprsNeedToDecode.size() > 0) {
+      val updatedAttrs = scan.attributesRaw.map(attr =>
+        updateDataType(attr.asInstanceOf[AttributeReference], relation, projectExprsNeedToDecode))
+      scan.attributesRaw = updatedAttrs
+      if (projectExprsNeedToDecode.size() > 0
+          && isDictionaryEncoded(projectExprsNeedToDecode.asScala.toSeq, relation)) {
         val decoder = getCarbonDecoder(logicalRelation,
           sc,
           tableName,
@@ -151,7 +158,12 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
           decoder
         }
       } else {
-        scan
+        if (scan.unprocessedExprs.nonEmpty) {
+          val filterCondToAdd = scan.unprocessedExprs.reduceLeftOption(expressions.And)
+          filterCondToAdd.map(Filter(_, scan)).getOrElse(scan)
+        } else {
+          scan
+        }
       }
     }
 
@@ -174,6 +186,31 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
         CarbonAliasDecoderRelation(), scan)(sc)
     }
 
+    def isDictionaryEncoded(projectExprsNeedToDecode: Seq[Attribute],
+        relation: CarbonDatasourceRelation): Boolean = {
+      var isEncoded = false
+      projectExprsNeedToDecode.foreach { attr =>
+        if (relation.carbonRelation.metaData.dictionaryMap.get(attr.name).getOrElse(false)) {
+          isEncoded = true
+        }
+      }
+      isEncoded
+    }
+
+    def updateDataType(attr: AttributeReference,
+        relation: CarbonDatasourceRelation,
+        allAttrsNotDecode: util.Set[Attribute]): AttributeReference = {
+      if (relation.carbonRelation.metaData.dictionaryMap.get(attr.name).getOrElse(false) &&
+        !allAttrsNotDecode.asScala.exists(p => p.name.equals(attr.name))) {
+        AttributeReference(attr.name,
+          IntegerType,
+          attr.nullable,
+          attr.metadata)(attr.exprId, attr.qualifiers)
+      } else {
+        attr
+      }
+    }
+
     private def isStarQuery(plan: LogicalPlan) = {
       plan match {
         case LogicalFilter(condition,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
index e121214..368c83b 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/datacompaction/DataCompactionLockTest.scala
@@ -2,18 +2,18 @@ package org.carbondata.spark.testsuite.datacompaction
 
 import java.io.File
 
-import org.apache.spark.sql.Row
+import scala.collection.JavaConverters._
+
 import org.apache.spark.sql.common.util.CarbonHiveContext._
 import org.apache.spark.sql.common.util.QueryTest
+import org.scalatest.BeforeAndAfterAll
+
 import org.carbondata.core.carbon.path.{CarbonStorePath, CarbonTablePath}
 import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifier}
 import org.carbondata.core.constants.CarbonCommonConstants
-import org.carbondata.core.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.carbondata.core.util.CarbonProperties
+import org.carbondata.lcm.locks.{CarbonLockFactory, ICarbonLock, LockUsage}
 import org.carbondata.lcm.status.SegmentStatusManager
-import org.scalatest.BeforeAndAfterAll
-
-import scala.collection.JavaConverters._
 
 /**
   * FT for data compaction Locking scenario.

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
index 8098308..c20e18d 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnGroupDataTypesTestCase.scala
@@ -47,37 +47,37 @@ class ColumnGroupDataTypesTestCase extends QueryTest with BeforeAndAfterAll {
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from colgrp"),
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from normal"))
   }
-  
+
   test("select all dimension query with filter on columnar") {
     checkAnswer(
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from colgrp where column1='column1666'"),
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from normal where column1='column1666'"))
   }
-  
+
   test("select all dimension query with filter on column group dimension") {
     checkAnswer(
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from colgrp where column3='column311'"),
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from normal where column3='column311'"))
   }
-  
+
   test("select all dimension query with filter on two dimension from different column group") {
     checkAnswer(
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from colgrp where column3='column311' and column7='column74' "),
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from normal where column3='column311' and column7='column74'"))
   }
-  
+
   test("select all dimension query with filter on two dimension from same column group") {
     checkAnswer(
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from colgrp where column3='column311' and column4='column42' "),
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from normal where column3='column311' and column4='column42'"))
   }
-  
+
   test("select all dimension query with filter on two dimension one from column group other from columnar") {
     checkAnswer(
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from colgrp where column3='column311' and column5='column516' "),
       sql("select column1,column2,column3,column4,column5,column6,column7,column8,column9,column10 from normal where column3='column311' and column5='column516'"))
   }
-  
+
   test("select few dimension") {
     checkAnswer(
       sql("select column1,column3,column4,column5,column6,column9,column10 from colgrp"),

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 3dd64be..de35082 100644
--- a/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
+++ b/processing/src/main/java/org/carbondata/processing/mdkeygen/MDKeyGenStep.java
@@ -39,7 +39,6 @@ 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.columnar.ColumnGroupModel;
 import org.carbondata.core.keygenerator.KeyGenException;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/4444c324/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 f1c7ad5..e9e1a56 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
@@ -54,8 +54,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.ByteUtil;
 import org.carbondata.core.util.CarbonMergerUtil;
@@ -67,6 +65,8 @@ import org.carbondata.core.writer.CarbonIndexFileWriter;
 import org.carbondata.format.BlockIndex;
 import org.carbondata.format.FileFooter;
 import org.carbondata.format.IndexHeader;
+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;
 


[08/50] [abbrv] incubator-carbondata git commit: Fixed driver btree performance issue (#729)

Posted by ch...@apache.org.
Fixed driver btree performance issue (#729)

LGTM

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

Branch: refs/heads/master
Commit: 82332b0e4381ede8a39789ccf1d917af0c89ab39
Parents: be46423
Author: Kumar Vishal <ku...@gmail.com>
Authored: Sat Jun 25 16:05:07 2016 +0800
Committer: sujith71955 <su...@gmail.com>
Committed: Sat Jun 25 16:05:07 2016 +0800

----------------------------------------------------------------------
 .../carbon/datastore/SegmentTaskIndexStore.java |  21 ++-
 .../carbon/metadata/index/BlockIndexInfo.java   |  92 ++++++++++++
 .../core/carbon/path/CarbonTablePath.java       |  74 +++++++---
 .../core/reader/CarbonIndexFileReader.java      |  95 ++++++++++++
 .../carbondata/core/reader/ThriftReader.java    |  47 ++++--
 .../core/util/CarbonMetadataUtil.java           |  78 ++++++++--
 .../org/carbondata/core/util/CarbonUtil.java    |  35 +++++
 .../core/writer/CarbonIndexFileWriter.java      |  64 +++++++++
 .../query/util/DataFileFooterConverter.java     |  44 ++++++
 .../datastore/SegmentTaskIndexStoreTest.java    | 143 -------------------
 format/src/main/thrift/carbondataindex.thrift   |  45 ++++++
 .../org/carbondata/hadoop/CarbonPathFilter.java |   4 +-
 .../store/writer/AbstractFactDataWriter.java    | 135 ++++++++++++++---
 13 files changed, 653 insertions(+), 224 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/main/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java b/core/src/main/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
index fde062c..28a892e 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStore.java
@@ -116,6 +116,7 @@ public class SegmentTaskIndexStore {
         addTableSegmentMap(absoluteTableIdentifier);
     Map<String, AbstractIndex> taskIdToSegmentIndexMap = null;
     String segmentId = null;
+    String taskId = null;
     try {
       while (iteratorOverSegmentBlocksInfos.hasNext()) {
         // segment id to table block mapping
@@ -147,8 +148,9 @@ public class SegmentTaskIndexStore {
                   taskIdToTableBlockInfoMap.entrySet().iterator();
               while (iterator.hasNext()) {
                 Entry<String, List<TableBlockInfo>> taskToBlockInfoList = iterator.next();
-                taskIdToSegmentIndexMap
-                    .put(taskToBlockInfoList.getKey(), loadBlocks(taskToBlockInfoList.getValue()));
+                taskId = taskToBlockInfoList.getKey();
+                taskIdToSegmentIndexMap.put(taskId,
+                    loadBlocks(taskId, taskToBlockInfoList.getValue(), absoluteTableIdentifier));
               }
               // removing from segment lock map as once segment is loaded
               //if concurrent query is coming for same segment
@@ -231,19 +233,12 @@ public class SegmentTaskIndexStore {
    * @return loaded segment
    * @throws CarbonUtilException
    */
-  private AbstractIndex loadBlocks(List<TableBlockInfo> tableBlockInfoList)
-      throws CarbonUtilException {
-    DataFileFooter footer = null;
+  private AbstractIndex loadBlocks(String taskId, List<TableBlockInfo> tableBlockInfoList,
+      AbsoluteTableIdentifier tableIdentifier) throws CarbonUtilException {
     // all the block of one task id will be loaded together
     // so creating a list which will have all the data file meta data to of one task
-    List<DataFileFooter> footerList = new ArrayList<DataFileFooter>();
-    for (TableBlockInfo tableBlockInfo : tableBlockInfoList) {
-      footer = CarbonUtil
-          .readMetadatFile(tableBlockInfo.getFilePath(), tableBlockInfo.getBlockOffset(),
-              tableBlockInfo.getBlockLength());
-      footer.setTableBlockInfo(tableBlockInfo);
-      footerList.add(footer);
-    }
+    List<DataFileFooter> footerList =
+        CarbonUtil.readCarbonIndexFile(taskId, tableBlockInfoList, tableIdentifier);
     AbstractIndex segment = new SegmentTaskIndex();
     // file path of only first block is passed as it all table block info path of
     // same task id will be same

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/main/java/org/carbondata/core/carbon/metadata/index/BlockIndexInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/index/BlockIndexInfo.java b/core/src/main/java/org/carbondata/core/carbon/metadata/index/BlockIndexInfo.java
new file mode 100644
index 0000000..bfed3dd
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/index/BlockIndexInfo.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.carbondata.core.carbon.metadata.index;
+
+import org.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
+
+/**
+ * Below class will be used hold the information
+ * about block index
+ */
+public class BlockIndexInfo {
+
+  /**
+   * total number of rows present in the file
+   */
+  private long numberOfRows;
+
+  /**
+   * file name
+   */
+  private String fileName;
+
+  /**
+   * offset of metadata in data file
+   */
+  private long offset;
+
+  /**
+   * to store min max and start and end key
+   */
+  private BlockletIndex blockletIndex;
+
+  /**
+   * Constructor
+   *
+   * @param numberOfRows  number of rows
+   * @param fileName      full qualified name
+   * @param offset        offset of the metadata in data file
+   * @param blockletIndex block let index
+   */
+  public BlockIndexInfo(long numberOfRows, String fileName, long offset,
+      BlockletIndex blockletIndex) {
+    this.numberOfRows = numberOfRows;
+    this.fileName = fileName;
+    this.offset = offset;
+    this.blockletIndex = blockletIndex;
+  }
+
+  /**
+   * @return the numberOfRows
+   */
+  public long getNumberOfRows() {
+    return numberOfRows;
+  }
+
+  /**
+   * @return the fileName
+   */
+  public String getFileName() {
+    return fileName;
+  }
+
+  /**
+   * @return the offset
+   */
+  public long getOffset() {
+    return offset;
+  }
+
+  /**
+   * @return the blockletIndex
+   */
+  public BlockletIndex getBlockletIndex() {
+    return blockletIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
index 31bc464..8dcd207 100644
--- a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
+++ b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
@@ -22,11 +22,14 @@ import java.io.File;
 
 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 static org.carbondata.core.constants.CarbonCommonConstants.INVALID_SEGMENT_ID;
 
 import org.apache.hadoop.fs.Path;
 
+
 /**
  * Helps to get Table content paths.
  */
@@ -44,6 +47,7 @@ public class CarbonTablePath extends Path {
   private static final String PARTITION_PREFIX = "Part";
   private static final String CARBON_DATA_EXT = ".carbondata";
   private static final String DATA_PART_PREFIX = "part";
+  private static final String INDEX_FILE_EXT = ".carbonindex";
 
   private String tablePath;
 
@@ -62,13 +66,6 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * gets table path
-   */
-  public String getPath() {
-    return tablePath;
-  }
-
-  /**
    * @param columnId unique column identifier
    * @return name of dictionary file
    */
@@ -78,6 +75,7 @@ public class CarbonTablePath extends Path {
 
   /**
    * whether carbonFile is dictionary file or not
+   *
    * @param carbonFile
    * @return
    */
@@ -86,6 +84,27 @@ public class CarbonTablePath extends Path {
   }
 
   /**
+   * check if it is carbon data file matching extension
+   *
+   * @param fileNameWithPath
+   * @return boolean
+   */
+  public static boolean isCarbonDataFile(String fileNameWithPath) {
+    int pos = fileNameWithPath.lastIndexOf('.');
+    if (pos != -1) {
+      return fileNameWithPath.substring(pos).startsWith(CARBON_DATA_EXT);
+    }
+    return false;
+  }
+
+  /**
+   * gets table path
+   */
+  public String getPath() {
+    return tablePath;
+  }
+
+  /**
    * @param columnId unique column identifier
    * @return absolute path of dictionary file
    */
@@ -148,6 +167,29 @@ public class CarbonTablePath extends Path {
   }
 
   /**
+   * Below method will be used to get the index file present in the segment folder
+   * based on task id
+   *
+   * @param taskId      task id of the file
+   * @param partitionId partition number
+   * @param segmentId   segment number
+   * @return full qualified carbon index path
+   */
+  public String getCarbonIndexFilePath(final String taskId, final String partitionId,
+      final String segmentId) {
+    String segmentDir = getSegmentDir(partitionId, segmentId);
+    CarbonFile carbonFile =
+        FileFactory.getCarbonFile(segmentDir, FileFactory.getFileType(segmentDir));
+
+    CarbonFile[] files = carbonFile.listFiles(new CarbonFileFilter() {
+      @Override public boolean accept(CarbonFile file) {
+        return file.getName().startsWith(taskId) && file.getName().endsWith(INDEX_FILE_EXT);
+      }
+    });
+    return files[0].getAbsolutePath();
+  }
+
+  /**
    * Gets absolute path of data file
    *
    * @param partitionId unique partition identifier
@@ -189,16 +231,14 @@ public class CarbonTablePath extends Path {
   }
 
   /**
-   * check if it is carbon data file matching extension
-   * @param fileNameWithPath
-   * @return boolean
+   * Below method will be used to get the carbon index filename
+   *
+   * @param taskNo               task number
+   * @param factUpdatedTimeStamp time stamp
+   * @return filename
    */
-  public static boolean isCarbonDataFile(String fileNameWithPath) {
-    int pos = fileNameWithPath.lastIndexOf('.');
-    if( pos != -1 ) {
-      return fileNameWithPath.substring(pos).startsWith(CARBON_DATA_EXT);
-    }
-    return false;
+  public String getCarbonIndexFileName(int taskNo, String factUpdatedTimeStamp) {
+    return taskNo + "-" + factUpdatedTimeStamp + INDEX_FILE_EXT;
   }
 
   private String getSegmentDir(String partitionId, String segmentId) {
@@ -234,7 +274,7 @@ public class CarbonTablePath extends Path {
     if (!(o instanceof CarbonTablePath)) {
       return false;
     }
-    CarbonTablePath path = (CarbonTablePath)o;
+    CarbonTablePath path = (CarbonTablePath) o;
     return tablePath.equals(path.tablePath) && super.equals(o);
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/main/java/org/carbondata/core/reader/CarbonIndexFileReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/CarbonIndexFileReader.java b/core/src/main/java/org/carbondata/core/reader/CarbonIndexFileReader.java
new file mode 100644
index 0000000..bb18e9f
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/reader/CarbonIndexFileReader.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.core.reader;
+
+import java.io.IOException;
+
+import org.carbondata.format.BlockIndex;
+import org.carbondata.format.IndexHeader;
+
+import org.apache.thrift.TBase;
+
+/**
+ * Reader class which will be used to read the index file
+ */
+public class CarbonIndexFileReader {
+
+  /**
+   * reader
+   */
+  private ThriftReader thriftReader;
+
+  /**
+   * Below method will be used to read the index header
+   *
+   * @return index header
+   * @throws IOException if any problem  while reader the header
+   */
+  public IndexHeader readIndexHeader() throws IOException {
+    IndexHeader indexHeader = (IndexHeader) thriftReader.read(new ThriftReader.TBaseCreator() {
+      @Override public TBase create() {
+        return new IndexHeader();
+      }
+    });
+    return indexHeader;
+  }
+
+  /**
+   * Below method will be used to close the reader
+   */
+  public void closeThriftReader() {
+    thriftReader.close();
+  }
+
+  /**
+   * Below method will be used to read the block index from fie
+   *
+   * @return block index info
+   * @throws IOException if problem while reading the block index
+   */
+  public BlockIndex readBlockIndexInfo() throws IOException {
+    BlockIndex blockInfo = (BlockIndex) thriftReader.read(new ThriftReader.TBaseCreator() {
+      @Override public TBase create() {
+        return new BlockIndex();
+      }
+    });
+    return blockInfo;
+  }
+
+  /**
+   * Open the thrift reader
+   *
+   * @param filePath
+   * @throws IOException
+   */
+  public void openThriftReader(String filePath) throws IOException {
+    thriftReader = new ThriftReader(filePath);
+    thriftReader.open();
+  }
+
+  /**
+   * check if any more object is present
+   *
+   * @return true if any more object can be read
+   * @throws IOException
+   */
+  public boolean hasNext() throws IOException {
+    return thriftReader.hasNext();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/main/java/org/carbondata/core/reader/ThriftReader.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/ThriftReader.java b/core/src/main/java/org/carbondata/core/reader/ThriftReader.java
index 3683a2b..e659919 100644
--- a/core/src/main/java/org/carbondata/core/reader/ThriftReader.java
+++ b/core/src/main/java/org/carbondata/core/reader/ThriftReader.java
@@ -35,34 +35,21 @@ import org.apache.thrift.transport.TIOStreamTransport;
  */
 public class ThriftReader {
   /**
-   * Thrift deserializes by taking an existing object and populating it. ThriftReader
-   * needs a way of obtaining instances of the class to be populated and this interface
-   * defines the mechanism by which a client provides these instances.
-   */
-  public static interface TBaseCreator {
-    TBase create();
-  }
-
-  /**
    * buffer size
    */
   private static final int bufferSize = 2048;
-
   /**
    * File containing the objects.
    */
   private String fileName;
-
   /**
    * Used to create empty objects that will be initialized with values from the fileName.
    */
-  private final TBaseCreator creator;
-
+  private TBaseCreator creator;
   /**
    * For reading the fileName.
    */
   private DataInputStream dataInputStream;
-
   /**
    * For reading the binary thrift objects.
    */
@@ -77,6 +64,13 @@ public class ThriftReader {
   }
 
   /**
+   * Constructor.
+   */
+  public ThriftReader(String fileName) {
+    this.fileName = fileName;
+  }
+
+  /**
    * Opens the fileName for reading.
    */
   public void open() throws IOException {
@@ -118,9 +112,34 @@ public class ThriftReader {
   }
 
   /**
+   * Reads the next object from the fileName.
+   *
+   * @param creator type of object which will be returned
+   * @throws IOException any problem while reading
+   */
+  public TBase read(TBaseCreator creator) throws IOException {
+    TBase t = creator.create();
+    try {
+      t.read(binaryIn);
+    } catch (TException e) {
+      throw new IOException(e);
+    }
+    return t;
+  }
+
+  /**
    * Close the fileName.
    */
   public void close() {
     CarbonUtil.closeStreams(dataInputStream);
   }
+
+  /**
+   * Thrift deserializes by taking an existing object and populating it. ThriftReader
+   * needs a way of obtaining instances of the class to be populated and this interface
+   * defines the mechanism by which a client provides these instances.
+   */
+  public static interface TBaseCreator {
+    TBase create();
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
index 5b9c291..5e8e8a2 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonMetadataUtil.java
@@ -11,10 +11,12 @@ import java.util.List;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.metadata.index.BlockIndexInfo;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.compression.ValueCompressionModel;
 import org.carbondata.core.metadata.BlockletInfoColumnar;
 import org.carbondata.core.metadata.ValueEncoderMeta;
+import org.carbondata.format.BlockIndex;
 import org.carbondata.format.BlockletBTreeIndex;
 import org.carbondata.format.BlockletIndex;
 import org.carbondata.format.BlockletInfo;
@@ -25,6 +27,7 @@ import org.carbondata.format.CompressionCodec;
 import org.carbondata.format.DataChunk;
 import org.carbondata.format.Encoding;
 import org.carbondata.format.FileFooter;
+import org.carbondata.format.IndexHeader;
 import org.carbondata.format.PresenceMeta;
 import org.carbondata.format.SegmentInfo;
 import org.carbondata.format.SortState;
@@ -68,6 +71,23 @@ public class CarbonMetadataUtil {
     return footer;
   }
 
+  private static BlockletIndex getBlockletIndex(
+      org.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex info) {
+    BlockletMinMaxIndex blockletMinMaxIndex = new BlockletMinMaxIndex();
+
+    for (int i = 0; i < info.getMinMaxIndex().getMaxValues().length; i++) {
+      blockletMinMaxIndex.addToMax_values(ByteBuffer.wrap(info.getMinMaxIndex().getMaxValues()[i]));
+      blockletMinMaxIndex.addToMin_values(ByteBuffer.wrap(info.getMinMaxIndex().getMinValues()[i]));
+    }
+    BlockletBTreeIndex blockletBTreeIndex = new BlockletBTreeIndex();
+    blockletBTreeIndex.setStart_key(info.getBtreeIndex().getStartKey());
+    blockletBTreeIndex.setEnd_key(info.getBtreeIndex().getEndKey());
+    BlockletIndex blockletIndex = new BlockletIndex();
+    blockletIndex.setMin_max_index(blockletMinMaxIndex);
+    blockletIndex.setB_tree_index(blockletBTreeIndex);
+    return blockletIndex;
+  }
+
   /**
    * Get total number of rows for the file.
    *
@@ -334,21 +354,53 @@ public class CarbonMetadataUtil {
         min[j] = minMaxIndexList.getMin_values().get(j).array();
         max[j] = minMaxIndexList.getMax_values().get(j).array();
       }
-
-      //      byte[][] min = new byte[minMaxIndexList.getMin_values().size()][];
-      //      List<ByteBuffer> minValues = minMaxIndexList.getMin_values();
-      //      for (int j = 0; j < minValues.size(); j++) {
-      //        min[j] = minValues.get(j).array();
-      //      }
-      //      listOfNodeInfo.get(i).setColumnMinData(min);
-      //
-      //      byte[][] max = new byte[minMaxIndexList.getMax_values().size()][];
-      //      List<ByteBuffer> maxValues = minMaxIndexList.getMax_values();
-      //      for (int j = 0; j < maxValues.size(); j++) {
-      //        max[j] = maxValues.get(j).array();
-      //    }
       listOfNodeInfo.get(i).setColumnMaxData(max);
     }
   }
 
+  /**
+   * Below method will be used to get the index header
+   *
+   * @param columnCardinality cardinality of each column
+   * @param columnSchemaList  list of column present in the table
+   * @return Index header object
+   */
+  public static IndexHeader getIndexHeader(int[] columnCardinality,
+      List<ColumnSchema> columnSchemaList) {
+    // create segment info object
+    SegmentInfo segmentInfo = new SegmentInfo();
+    // set the number of columns
+    segmentInfo.setNum_cols(columnSchemaList.size());
+    // setting the column cardinality
+    segmentInfo.setColumn_cardinalities(CarbonUtil.convertToIntegerList(columnCardinality));
+    // create index header object
+    IndexHeader indexHeader = new IndexHeader();
+    // set the segment info
+    indexHeader.setSegment_info(segmentInfo);
+    // set the column names
+    indexHeader.setTable_columns(columnSchemaList);
+    return indexHeader;
+  }
+
+  /**
+   * Below method will be used to get the block index info thrift object for each block
+   * present in the segment
+   *
+   * @param blockIndexInfoList block index info list
+   * @return list of block index
+   */
+  public static List<BlockIndex> getBlockIndexInfo(List<BlockIndexInfo> blockIndexInfoList) {
+    List<BlockIndex> thriftBlockIndexList = new ArrayList<BlockIndex>();
+    BlockIndex blockIndex = null;
+    // below code to create block index info object for each block
+    for (BlockIndexInfo blockIndexInfo : blockIndexInfoList) {
+      blockIndex = new BlockIndex();
+      blockIndex.setNum_rows(blockIndexInfo.getNumberOfRows());
+      blockIndex.setOffset(blockIndexInfo.getNumberOfRows());
+      blockIndex.setFile_name(blockIndexInfo.getFileName());
+      blockIndex.setBlock_index(getBlockletIndex(blockIndexInfo.getBlockletIndex()));
+      thriftBlockIndexList.add(blockIndex);
+    }
+    return thriftBlockIndexList;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/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 58e6f42..427fb97 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -48,12 +48,16 @@ import java.util.concurrent.Executors;
 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.block.TableBlockInfo;
 import org.carbondata.core.carbon.datastore.chunk.impl.FixedLengthDimensionDataChunk;
 import org.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
 import org.carbondata.core.carbon.metadata.blocklet.datachunk.DataChunk;
 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.path.CarbonStorePath;
+import org.carbondata.core.carbon.path.CarbonTablePath;
 import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.datastorage.store.FileHolder;
 import org.carbondata.core.datastorage.store.columnar.ColumnarKeyStoreDataHolder;
@@ -1683,5 +1687,36 @@ public final class CarbonUtil {
     }
   }
 
+  /**
+   * Below method will be used to get all the block index info from index file
+   *
+   * @param taskId                  task id of the file
+   * @param tableBlockInfoList      list of table block
+   * @param absoluteTableIdentifier absolute table identifier
+   * @return list of block info
+   * @throws CarbonUtilException if any problem while reading
+   */
+  public static List<DataFileFooter> readCarbonIndexFile(String taskId,
+      List<TableBlockInfo> tableBlockInfoList, AbsoluteTableIdentifier absoluteTableIdentifier)
+      throws CarbonUtilException {
+    // need to sort the  block info list based for task in ascending  order so
+    // it will be sinkup with block index read from file
+    Collections.sort(tableBlockInfoList);
+    CarbonTablePath carbonTablePath = CarbonStorePath
+        .getCarbonTablePath(absoluteTableIdentifier.getStorePath(),
+            absoluteTableIdentifier.getCarbonTableIdentifier());
+    // geting the index file path
+    //TODO need to pass proper partition number when partiton will be supported
+    String carbonIndexFilePath = carbonTablePath
+        .getCarbonIndexFilePath(taskId, "0", tableBlockInfoList.get(0).getSegmentId());
+    DataFileFooterConverter fileFooterConverter = new DataFileFooterConverter();
+    try {
+      // read the index info and return
+      return fileFooterConverter.getIndexInfo(carbonIndexFilePath, tableBlockInfoList);
+    } catch (IOException e) {
+      throw new CarbonUtilException("Problem while reading the file metadata", e);
+    }
+  }
+
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/main/java/org/carbondata/core/writer/CarbonIndexFileWriter.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/CarbonIndexFileWriter.java b/core/src/main/java/org/carbondata/core/writer/CarbonIndexFileWriter.java
new file mode 100644
index 0000000..5ae7b33
--- /dev/null
+++ b/core/src/main/java/org/carbondata/core/writer/CarbonIndexFileWriter.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.core.writer;
+
+import java.io.IOException;
+
+import org.apache.thrift.TBase;
+
+/**
+ * Reader class which will be used to read the index file
+ */
+public class CarbonIndexFileWriter {
+
+  /**
+   * thrift writer object
+   */
+  private ThriftWriter thriftWriter;
+
+  /**
+   * It writes thrift object to file
+   *
+   * @param footer
+   * @throws IOException
+   */
+  public void writeThrift(TBase indexObject) throws IOException {
+    thriftWriter.write(indexObject);
+  }
+
+  /**
+   * Below method will be used to open the thrift writer
+   *
+   * @param filePath file path where data need to be written
+   * @throws IOException throws io exception in case of any failure
+   */
+  public void openThriftWriter(String filePath) throws IOException {
+    // create thrift writer instance
+    thriftWriter = new ThriftWriter(filePath, true);
+    // open the file stream
+    thriftWriter.open();
+  }
+
+  /**
+   * Below method will be used to close the thrift object
+   */
+  public void close() {
+    thriftWriter.close();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/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
index 7ca9a3f..fc23a0e 100644
--- a/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java
@@ -29,6 +29,7 @@ import java.util.List;
 
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
+import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.carbondata.core.carbon.metadata.blocklet.BlockletInfo;
 import org.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
 import org.carbondata.core.carbon.metadata.blocklet.SegmentInfo;
@@ -47,8 +48,10 @@ 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.reader.CarbonIndexFileReader;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.format.BlockIndex;
 import org.carbondata.format.FileFooter;
 
 /**
@@ -61,6 +64,47 @@ public class DataFileFooterConverter {
       LogServiceFactory.getLogService(DataFileFooterConverter.class.getName());
 
   /**
+   * Below method will be used to get the index info from index file
+   *
+   * @param filePath           file path of the index file
+   * @param tableBlockInfoList table block index
+   * @return list of index info
+   * @throws IOException problem while reading the index file
+   */
+  public List<DataFileFooter> getIndexInfo(String filePath, List<TableBlockInfo> tableBlockInfoList)
+      throws IOException {
+    CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
+    // open the reader
+    indexReader.openThriftReader(filePath);
+    // get the index header
+    org.carbondata.format.IndexHeader readIndexHeader = indexReader.readIndexHeader();
+    List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
+    List<org.carbondata.format.ColumnSchema> table_columns = readIndexHeader.getTable_columns();
+    for (int i = 0; i < table_columns.size(); i++) {
+      columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
+    }
+    // get the segment info
+    SegmentInfo segmentInfo = getSegmentInfo(readIndexHeader.getSegment_info());
+    BlockletIndex blockletIndex = null;
+    int counter = 0;
+    DataFileFooter dataFileFooter = null;
+    List<DataFileFooter> dataFileFooters = new ArrayList<DataFileFooter>();
+    // read the block info from file
+    while (indexReader.hasNext()) {
+      BlockIndex readBlockIndexInfo = indexReader.readBlockIndexInfo();
+      blockletIndex = getBlockletIndex(readBlockIndexInfo.getBlock_index());
+      dataFileFooter = new DataFileFooter();
+      dataFileFooter.setBlockletIndex(blockletIndex);
+      dataFileFooter.setColumnInTable(columnSchemaList);
+      dataFileFooter.setNumberOfRows(readBlockIndexInfo.getNum_rows());
+      dataFileFooter.setTableBlockInfo(tableBlockInfoList.get(counter++));
+      dataFileFooter.setSegmentInfo(segmentInfo);
+      dataFileFooters.add(dataFileFooter);
+    }
+    return dataFileFooters;
+  }
+
+  /**
    * Below method will be used to convert thrift file meta to wrapper file meta
    */
   public DataFileFooter readDataFileFooter(String filePath, long blockOffset, long blockLength)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/core/src/test/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStoreTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStoreTest.java b/core/src/test/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStoreTest.java
deleted file mode 100644
index 328917d..0000000
--- a/core/src/test/java/org/carbondata/core/carbon/datastore/SegmentTaskIndexStoreTest.java
+++ /dev/null
@@ -1,143 +0,0 @@
-package org.carbondata.core.carbon.datastore;
-
-import java.io.File;
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.TimeUnit;
-
-import org.carbondata.core.carbon.AbsoluteTableIdentifier;
-import org.carbondata.core.carbon.CarbonTableIdentifier;
-import org.carbondata.core.carbon.datastore.block.AbstractIndex;
-import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
-import org.carbondata.core.carbon.datastore.exception.IndexBuilderException;
-
-import junit.framework.TestCase;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-public class SegmentTaskIndexStoreTest extends TestCase {
-
-  private SegmentTaskIndexStore indexStore;
-
-  @BeforeClass public void setUp() {
-    indexStore = SegmentTaskIndexStore.getInstance();
-  }
-
-  @Test public void testloadAndGetTaskIdToSegmentsMapForSingleSegment() throws IOException {
-    String canonicalPath =
-        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-    File file = new File(canonicalPath + "/src/test/resources/part-0-0-1466029397000.carbondata");
-    TableBlockInfo info =
-        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
-    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-    Map<String, List<TableBlockInfo>> mapOfSegmentToTableBlockInfoList = new HashMap<>();
-    mapOfSegmentToTableBlockInfoList.put("0", Arrays.asList(new TableBlockInfo[] { info }));
-    Map<String, AbstractIndex> loadAndGetTaskIdToSegmentsMap = null;
-    try {
-      loadAndGetTaskIdToSegmentsMap = indexStore
-          .loadAndGetTaskIdToSegmentsMap(mapOfSegmentToTableBlockInfoList, absoluteTableIdentifier);
-    } catch (IndexBuilderException e) {
-      assertTrue(false);
-    }
-    assertTrue(loadAndGetTaskIdToSegmentsMap.size() == 1);
-    indexStore.removeTableBlocks(Arrays.asList(new String[] { "0" }), absoluteTableIdentifier);
-  }
-
-  //@Test
-  public void testloadAndGetTaskIdToSegmentsMapForSameSegmentLoadedConcurrently()
-      throws IOException {
-    String canonicalPath =
-        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-    File file = new File(canonicalPath + "/src/test/resources/part-0-0-1466029397000.carbondata");
-    TableBlockInfo info =
-        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
-    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-    Map<String, List<TableBlockInfo>> mapOfSegmentToTableBlockInfoList = new HashMap<>();
-    mapOfSegmentToTableBlockInfoList.put("0", Arrays.asList(new TableBlockInfo[] { info }));
-    ExecutorService executor = Executors.newFixedThreadPool(2);
-
-    executor
-        .submit(new SegmentLoaderThread(mapOfSegmentToTableBlockInfoList, absoluteTableIdentifier));
-    executor
-        .submit(new SegmentLoaderThread(mapOfSegmentToTableBlockInfoList, absoluteTableIdentifier));
-    executor.shutdown();
-    try {
-      executor.awaitTermination(1, TimeUnit.DAYS);
-    } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-    assertTrue(indexStore.getSegmentBTreeIfExists(absoluteTableIdentifier, "0").size() == 1);
-    indexStore.removeTableBlocks(Arrays.asList(new String[] { "0" }), absoluteTableIdentifier);
-  }
-
-  @Test public void testloadAndGetTaskIdToSegmentsMapForDifferentSegmentLoadedConcurrently()
-      throws IOException {
-    String canonicalPath =
-        new File(this.getClass().getResource("/").getPath() + "/../../").getCanonicalPath();
-    File file = new File(canonicalPath + "/src/test/resources/part-0-0-1466029397000.carbondata");
-    TableBlockInfo info =
-        new TableBlockInfo(file.getAbsolutePath(), 0, "0", new String[] { "loclhost" },
-            file.length());
-    TableBlockInfo info1 =
-        new TableBlockInfo(file.getAbsolutePath(), 0, "1", new String[] { "loclhost" },
-            file.length());
-    CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("default", "t3", "1");
-    AbsoluteTableIdentifier absoluteTableIdentifier =
-        new AbsoluteTableIdentifier("/src/test/resources", carbonTableIdentifier);
-    Map<String, List<TableBlockInfo>> mapOfSegmentToTableBlockInfoList = new HashMap<>();
-    Map<String, List<TableBlockInfo>> mapOfSegmentToTableBlockInfoList1 = new HashMap<>();
-    mapOfSegmentToTableBlockInfoList.put("0", Arrays.asList(new TableBlockInfo[] { info }));
-    mapOfSegmentToTableBlockInfoList1.put("1", Arrays.asList(new TableBlockInfo[] { info1 }));
-    ExecutorService executor = Executors.newFixedThreadPool(2);
-
-    executor
-        .submit(new SegmentLoaderThread(mapOfSegmentToTableBlockInfoList, absoluteTableIdentifier));
-    executor.submit(
-        new SegmentLoaderThread(mapOfSegmentToTableBlockInfoList1, absoluteTableIdentifier));
-
-    executor.shutdown();
-    try {
-      executor.awaitTermination(1, TimeUnit.DAYS);
-    } catch (InterruptedException e) {
-      // TODO Auto-generated catch block
-      e.printStackTrace();
-    }
-    assertTrue(indexStore.getSegmentBTreeIfExists(absoluteTableIdentifier, "0").size() == 1);
-    assertTrue(indexStore.getSegmentBTreeIfExists(absoluteTableIdentifier, "1").size() == 1);
-    indexStore.removeTableBlocks(Arrays.asList(new String[] { "0" }), absoluteTableIdentifier);
-    indexStore.removeTableBlocks(Arrays.asList(new String[] { "1" }), absoluteTableIdentifier);
-  }
-
-  private class SegmentLoaderThread implements Callable<Void> {
-    private Map<String, List<TableBlockInfo>> mapOfSegmentToTableBlockInfoList;
-
-    private AbsoluteTableIdentifier absoluteTableIdentifier;
-
-    public SegmentLoaderThread(Map<String, List<TableBlockInfo>> mapOfSegmentToTableBlockInfoList,
-        AbsoluteTableIdentifier absoluteTableIdentifier) {
-      // TODO Auto-generated constructor stub
-      this.mapOfSegmentToTableBlockInfoList = mapOfSegmentToTableBlockInfoList;
-      this.absoluteTableIdentifier = absoluteTableIdentifier;
-    }
-
-    @Override public Void call() throws Exception {
-      indexStore
-          .loadAndGetTaskIdToSegmentsMap(mapOfSegmentToTableBlockInfoList, absoluteTableIdentifier);
-      return null;
-    }
-
-  }
-}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/format/src/main/thrift/carbondataindex.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/carbondataindex.thrift b/format/src/main/thrift/carbondataindex.thrift
new file mode 100644
index 0000000..14159f1
--- /dev/null
+++ b/format/src/main/thrift/carbondataindex.thrift
@@ -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.
+ */
+
+/**
+ * File format description for the carbon file format
+ */
+namespace java org.carbondata.format
+
+include "schema.thrift"
+include "carbondata.thrift"
+
+/**
+ * header information stored in index file
+ */
+struct IndexHeader{
+  1: required i32 version; // version used for data compatibility
+  2: required list<schema.ColumnSchema> table_columns;	// Description of columns in this file
+  3: required carbondata.SegmentInfo segment_info;	// Segment info (will be same/repeated for all files in this segment)
+}
+
+/**
+ * block index information stored in index file for every block
+ */
+struct BlockIndex{
+  1: required i64 num_rows; // Total number of rows in this file
+  2: required string file_name; // Block file name
+  3: required i64 offset; // Offset of block
+  4: required carbondata.BlockletIndex block_index;	// Block index
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/hadoop/src/main/java/org/carbondata/hadoop/CarbonPathFilter.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/CarbonPathFilter.java b/hadoop/src/main/java/org/carbondata/hadoop/CarbonPathFilter.java
index 0e310f3..1bdd3d1 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/CarbonPathFilter.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/CarbonPathFilter.java
@@ -18,6 +18,8 @@
  */
 package org.carbondata.hadoop;
 
+import org.carbondata.core.carbon.path.CarbonTablePath;
+
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.PathFilter;
 
@@ -37,6 +39,6 @@ public class CarbonPathFilter implements PathFilter {
   }
 
   @Override public boolean accept(Path path) {
-    return true;
+    return CarbonTablePath.isCarbonDataFile(path.getName());
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/82332b0e/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..4e49806 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
@@ -40,8 +40,12 @@ import java.util.concurrent.TimeUnit;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
+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.converter.SchemaConverter;
 import org.carbondata.core.carbon.metadata.converter.ThriftWrapperSchemaConverterImpl;
+import org.carbondata.core.carbon.metadata.index.BlockIndexInfo;
 import org.carbondata.core.carbon.metadata.schema.table.CarbonTable;
 import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 import org.carbondata.core.carbon.path.CarbonStorePath;
@@ -52,17 +56,20 @@ 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.ByteUtil;
 import org.carbondata.core.util.CarbonMergerUtil;
 import org.carbondata.core.util.CarbonMetadataUtil;
 import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.writer.CarbonFooterWriter;
+import org.carbondata.core.writer.CarbonIndexFileWriter;
+import org.carbondata.format.BlockIndex;
 import org.carbondata.format.FileFooter;
+import org.carbondata.format.IndexHeader;
 import org.carbondata.processing.store.CarbonDataFileAttributes;
 import org.carbondata.processing.store.writer.exception.CarbonDataWriterException;
 
 import org.apache.commons.lang3.ArrayUtils;
-
 import org.apache.hadoop.io.IOUtils;
 
 public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<T>
@@ -164,6 +171,8 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   private int spaceReservedForBlockMetaSize;
   private FileOutputStream fileOutputStream;
 
+  private List<BlockIndexInfo> blockIndexInfoList;
+
   public AbstractFactDataWriter(String storeLocation, int measureCount, int mdKeyLength,
       String databaseName, String tableName, IFileManagerComposite fileManager, int[] keyBlockSize,
       CarbonDataFileAttributes carbonDataFileAttributes, List<ColumnSchema> columnSchema,
@@ -180,6 +189,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     this.storeLocation = storeLocation;
     this.blockletInfoList =
         new ArrayList<BlockletInfoColumnar>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
+    blockIndexInfoList = new ArrayList<>();
     // get max file size;
     CarbonProperties propInstance = CarbonProperties.getInstance();
     this.fileSizeInBytes = Long.parseLong(propInstance
@@ -223,6 +233,27 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   }
 
   /**
+   * This method will return max of block size and file size
+   *
+   * @param blockSize
+   * @param fileSize
+   * @return
+   */
+  private static long getMaxOfBlockAndFileSize(long blockSize, long fileSize) {
+    long maxSize = blockSize;
+    if (fileSize > blockSize) {
+      maxSize = fileSize;
+    }
+    // block size should be exactly divisible by 512 which is  maintained by HDFS as bytes
+    // per checksum, dfs.bytes-per-checksum=512 must divide block size
+    long remainder = maxSize % HDFS_CHECKSUM_LENGTH;
+    if (remainder > 0) {
+      maxSize = maxSize + HDFS_CHECKSUM_LENGTH - remainder;
+    }
+    return maxSize;
+  }
+
+  /**
    * @param isNoDictionary the isNoDictionary to set
    */
   public void setIsNoDictionary(boolean[] isNoDictionary) {
@@ -237,7 +268,7 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
    * current file size to 0 close the existing file channel get the new file
    * name and get the channel for new file
    *
-   * @param blockletDataSize  data size of one block
+   * @param blockletDataSize data size of one block
    * @throws CarbonDataWriterException if any problem
    */
   protected void updateBlockletFileChannel(long blockletDataSize) throws CarbonDataWriterException {
@@ -326,11 +357,56 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
       FileFooter convertFileMeta = CarbonMetadataUtil
           .convertFileFooter(infoList, localCardinality.length, localCardinality,
               thriftColumnSchemaList);
+      fillBlockIndexInfoDetails(infoList, convertFileMeta.getNum_rows(), filePath, currentPosition);
       writer.writeFooter(convertFileMeta, currentPosition);
     } catch (IOException e) {
       throw new CarbonDataWriterException("Problem while writing the carbon file: ", e);
     }
+  }
 
+  /**
+   * Below method will be used to fill the vlock info details
+   *
+   * @param infoList        info list
+   * @param numberOfRows    number of rows in file
+   * @param filePath        file path
+   * @param currentPosition current offset
+   */
+  private void fillBlockIndexInfoDetails(List<BlockletInfoColumnar> infoList, long numberOfRows,
+      String filePath, long currentPosition) {
+
+    // as min-max will change for each blocklet and second blocklet min-max can be lesser than
+    // the first blocklet so we need to calculate the complete block level min-max by taking
+    // the min value of each column and max value of each column
+    byte[][] currentMinValue = infoList.get(0).getColumnMinData().clone();
+    byte[][] currentMaxValue = infoList.get(0).getColumnMaxData().clone();
+    byte[][] minValue = null;
+    byte[][] maxValue = null;
+    for (int i = 1; i < infoList.size(); i++) {
+      minValue = infoList.get(i).getColumnMinData();
+      maxValue = infoList.get(i).getColumnMaxData();
+      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();
+        }
+      }
+    }
+    // start and end key we can take based on first blocklet
+    // start key will be the block start key as
+    // it is the least key and end blocklet end key will be the block end key as it is the max key
+    BlockletBTreeIndex btree = new BlockletBTreeIndex(infoList.get(0).getStartKey(),
+        infoList.get(infoList.size() - 1).getEndKey());
+    BlockletMinMaxIndex minmax = new BlockletMinMaxIndex();
+    minmax.setMinValues(currentMinValue);
+    minmax.setMaxValues(currentMaxValue);
+    BlockletIndex blockletIndex = new BlockletIndex(btree, minmax);
+    BlockIndexInfo blockIndexInfo =
+        new BlockIndexInfo(numberOfRows, filePath.substring(0, filePath.lastIndexOf('.')),
+            currentPosition, blockletIndex);
+    blockIndexInfoList.add(blockIndexInfo);
   }
 
   protected List<org.carbondata.format.ColumnSchema> getColumnSchemaListAndCardinality(
@@ -422,10 +498,44 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
     CarbonUtil.closeStreams(this.fileOutputStream, this.fileChannel);
     renameCarbonDataFile();
     copyCarbonDataFileToCarbonStorePath(this.fileName.substring(0, this.fileName.lastIndexOf('.')));
+    try {
+      writeIndexFile();
+    } catch (IOException e) {
+      throw new CarbonDataWriterException("Problem while writing the index file", e);
+    }
     closeExecutorService();
   }
 
   /**
+   * Below method will be used to write the idex file
+   *
+   * @throws IOException               throws io exception if any problem while writing
+   * @throws CarbonDataWriterException data writing
+   */
+  private void writeIndexFile() throws IOException, CarbonDataWriterException {
+    // get the header
+    IndexHeader indexHeader =
+        CarbonMetadataUtil.getIndexHeader(localCardinality, thriftColumnSchemaList);
+    // get the block index info thrift
+    List<BlockIndex> blockIndexThrift = CarbonMetadataUtil.getBlockIndexInfo(blockIndexInfoList);
+    String fileName = storeLocation + File.separator + carbonTablePath
+        .getCarbonIndexFileName(carbonDataFileAttributes.getTaskId(),
+            carbonDataFileAttributes.getFactTimeStamp());
+    CarbonIndexFileWriter writer = new CarbonIndexFileWriter();
+    // open file
+    writer.openThriftWriter(fileName);
+    // write the header first
+    writer.writeThrift(indexHeader);
+    // write the indexes
+    for (BlockIndex blockIndex : blockIndexThrift) {
+      writer.writeThrift(blockIndex);
+    }
+    writer.close();
+    // copy from temp to actual store location
+    copyCarbonDataFileToCarbonStorePath(fileName);
+  }
+
+  /**
    * This method will close the executor service which is used for copying carbon
    * data files to carbon store path
    *
@@ -517,27 +627,6 @@ public abstract class AbstractFactDataWriter<T> implements CarbonFactDataWriter<
   }
 
   /**
-   * This method will return max of block size and file size
-   *
-   * @param blockSize
-   * @param fileSize
-   * @return
-   */
-  private static long getMaxOfBlockAndFileSize(long blockSize, long fileSize) {
-    long maxSize = blockSize;
-    if (fileSize > blockSize) {
-      maxSize = fileSize;
-    }
-    // block size should be exactly divisible by 512 which is  maintained by HDFS as bytes
-    // per checksum, dfs.bytes-per-checksum=512 must divide block size
-    long remainder = maxSize % HDFS_CHECKSUM_LENGTH;
-    if (remainder > 0) {
-      maxSize = maxSize + HDFS_CHECKSUM_LENGTH - remainder;
-    }
-    return maxSize;
-  }
-
-  /**
    * Write leaf meta data to File.
    *
    * @throws CarbonDataWriterException


[09/50] [abbrv] incubator-carbondata git commit: [BUG]Column group with no dictionary and timestamp (#713)

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e96de9f5/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..4c991f5 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
@@ -1,17 +1,20 @@
 package org.carbondata.processing.store.colgroup;
 
+import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.HashSet;
 import java.util.Iterator;
+import java.util.List;
 import java.util.Random;
 import java.util.Set;
+import java.util.UUID;
 
+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.ColumnSchema;
+import org.carbondata.core.constants.CarbonCommonConstants;
 import org.carbondata.core.keygenerator.KeyGenException;
-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;
@@ -24,206 +27,213 @@ import org.junit.Test;
  * RowStore store min max test
  */
 public class ColGroupMinMaxTest {
-    /**
-     * colgrpmodel
-     */
-    ColumnGroupModel colGrpModel;
-    /**
-     * column splitter
-     */
-    ColumnarSplitter columnSplitter;
-    /**
-     * column groups
-     */
-    int[][] columnGroups;
-
-    /**
-     * surrogate key
-     */
-    int[][] data;
-    /**
-     * mdkey data
-     */
-    byte[][] mdkeyData;
-
-    /**
-     * min value of surrogates
-     */
-    int[] min;
-    /**
-     * max value of surrogates
-     */
-    int[] max;
-    private ColGroupMinMax[] colGrpMinMax;
-
-    @Before
-    public void setupBeforeClass() throws KeyGenException {
-	int[] dimLens = new int[] { 100000, 1000, 10, 100, 100, 10000, 1000,
-		10, 1000, 1 };
-	columnGroups = new int[][] { { 0, 1, 2 }, { 3, 4 }, { 5, 6 },
-		{ 7, 8, 9 } };
-	colGrpModel = CarbonUtil.getColGroupModel(dimLens, columnGroups);
-	columnSplitter = new MultiDimKeyVarLengthVariableSplitGenerator(
-		CarbonUtil.getDimensionBitLength(
-			colGrpModel.getColumnGroupCardinality(),
-			colGrpModel.getColumnSplit()),
-		colGrpModel.getColumnSplit());
-	KeyGenerator keyGenerator = (KeyGenerator) columnSplitter;
-	initColGrpMinMax();
-	Random random = new Random();
-	data = new int[1000][];
-	min = new int[dimLens.length];
-	Arrays.fill(min, Integer.MAX_VALUE);
-	max = new int[dimLens.length];
-	Arrays.fill(max, Integer.MIN_VALUE);
-	for (int i = 0; i < 1000; i++) {
-
-	    data[i] = new int[dimLens.length];
-	    for (int j = 0; j < data[i].length; j++) {
-		data[i][j] = random.nextInt(dimLens[j]);
-	    }
-	    setMinData(data[i]);
-	    setMaxData(data[i]);
-	    System.out.println(Arrays.toString(data[i]));
-	}
-	mdkeyData = new byte[1000][];
-	for (int i = 0; i < 1000; i++) {
-	    mdkeyData[i] = keyGenerator.generateKey(data[i]);
-	    evaluateColGrpMinMax(mdkeyData[i]);
-	}
-    }
-
-    private void evaluateColGrpMinMax(byte[] mdkey) {
 
-	for (int colGrp = 0; colGrp < colGrpModel.getColumnGroup().length; colGrp++) {
-	    if (colGrpModel.getColumnGroup()[colGrp].length > 0) {
-		colGrpMinMax[colGrp].add(mdkey);
-	    }
+	/**
+	 * column groups
+	 */
+	int[][] columnGroups;
+
+	/**
+	 * surrogate key
+	 */
+	int[][] data;
+	/**
+	 * mdkey data
+	 */
+	byte[][] mdkeyData;
+
+	/**
+	 * min value of surrogates
+	 */
+	int[] min;
+	/**
+	 * max value of surrogates
+	 */
+	int[] max;
+	private ColGroupMinMax[] colGrpMinMax;
+	
+	private SegmentProperties segmentProperties;
+
+	@Before
+	public void setupBeforeClass() throws KeyGenException {
+		int[] dimLens = new int[] { 100000, 1000, 10, 100, 100, 10000, 1000, 10,
+				1000, 1 };
+		columnGroups = new int[][] { { 0, 1, 2 }, { 3, 4 }, { 5, 6 }, { 7, 8, 9 } };
+		segmentProperties = getSegmentProperties(dimLens, columnGroups);
+		initColGrpMinMax();
+		Random random = new Random();
+		data = new int[1000][];
+		min = new int[dimLens.length];
+		Arrays.fill(min, Integer.MAX_VALUE);
+		max = new int[dimLens.length];
+		Arrays.fill(max, Integer.MIN_VALUE);
+		for (int i = 0; i < 1000; i++) {
+
+			data[i] = new int[dimLens.length];
+			for (int j = 0; j < data[i].length; j++) {
+				data[i][j] = random.nextInt(dimLens[j]);
+			}
+			setMinData(data[i]);
+			setMaxData(data[i]);
+			System.out.println(Arrays.toString(data[i]));
+		}
+		mdkeyData = new byte[1000][];
+		for (int i = 0; i < 1000; i++) {
+			mdkeyData[i] = segmentProperties.getDimensionKeyGenerator().generateKey(data[i]);
+			evaluateColGrpMinMax(mdkeyData[i]);
+		}
 	}
-    }
-
-    private void initColGrpMinMax() {
-	int[][] colGrps = colGrpModel.getColumnGroup();
-	colGrpMinMax = new ColGroupMinMax[colGrps.length];
-	for (int colGrp = 0; colGrp < colGrps.length; colGrp++) {
-	    if (colGrps[colGrp].length > 0) {
-		colGrpMinMax[colGrp] = new ColGroupMinMax(colGrpModel,
-			columnSplitter, colGrp);
-	    }
-	}
-    }
 
-    private void setMaxData(int[] data) {
-	for (int i = 0; i < max.length; i++) {
-	    if (max[i] < data[i]) {
-		max[i] = data[i];
-	    }
+	private SegmentProperties getSegmentProperties(int[] dimLens, int[][] columnGroups) {
+		List<ColumnSchema> columnSchemas = new ArrayList<>();
+		for(int i=0;i<columnGroups.length;i++) {
+			  for(int j=0;j<columnGroups[i].length;j++) {
+			  	  columnSchemas.add(getDimensionColumn(i+j,i));
+			  }
+			
+		}
+		return new SegmentProperties(columnSchemas, dimLens);
 	}
+	private ColumnSchema getDimensionColumn(int var , int groupId) {
+  ColumnSchema dimColumn = new ColumnSchema();
+  dimColumn.setColumnar(false);
+  dimColumn.setColumnName("IMEI"+var);
+  dimColumn.setColumnUniqueId(UUID.randomUUID().toString());
+  dimColumn.setDataType(DataType.STRING);
+  dimColumn.setDimensionColumn(true);
+  List<Encoding> encodeList =
+      new ArrayList<Encoding>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  encodeList.add(Encoding.DICTIONARY);
+  dimColumn.setEncodingList(encodeList);
+  dimColumn.setColumnGroup(0);
+  dimColumn.setNumberOfChild(0);
+  return dimColumn;
+}
 
-    }
+	private void evaluateColGrpMinMax(byte[] mdkey) {
 
-    private void setMinData(int[] data) {
-	for (int i = 0; i < min.length; i++) {
-	    if (min[i] > data[i]) {
-		min[i] = data[i];
-	    }
+		for (int colGrp = 0; colGrp < segmentProperties.getColumnGroups().length; colGrp++) {
+			if (segmentProperties.getColumnGroups()[colGrp].length > 0) {
+				colGrpMinMax[colGrp].add(mdkey);
+			}
+		}
 	}
-    }
 
-    @Test
-    public void testRowStoreMinMax() throws KeyGenException {
+	private void initColGrpMinMax() {
+		int[][] colGrps = segmentProperties.getColumnGroups();
+		colGrpMinMax = new ColGroupMinMax[colGrps.length];
+		for (int colGrp = 0; colGrp < colGrps.length; colGrp++) {
+			if (colGrps[colGrp].length > 0) {
+				colGrpMinMax[colGrp] = new ColGroupMinMax(segmentProperties,
+						colGrp);
+			}
+		}
+	}
 
-	DataHolder[] dataHolders = getDataHolders(
-		colGrpModel.getNoOfColumnStore(), mdkeyData.length);
-	for (int i = 0; i < mdkeyData.length; i++) {
-	    byte[][] split = columnSplitter.splitKey(mdkeyData[i]);
-	    for (int j = 0; j < split.length; j++) {
-		dataHolders[j].addData(split[j], i);
-	    }
+	private void setMaxData(int[] data) {
+		for (int i = 0; i < max.length; i++) {
+			if (max[i] < data[i]) {
+				max[i] = data[i];
+			}
+		}
 
 	}
-	ColGroupBlockStorage[] rowBlockStorage = new ColGroupBlockStorage[dataHolders.length];
-	for (int i = 0; i < dataHolders.length; i++) {
 
-	    rowBlockStorage[i] = new ColGroupBlockStorage(dataHolders[i]);
-	}
-	int[][] columnGroup = colGrpModel.getColumnGroup();
-	for (int i = 0; i < dataHolders.length; i++) {
-	    assertMinMax(colGrpMinMax[i].getMin(), rowBlockStorage[i].getMax(),
-		    columnGroup[i]);
+	private void setMinData(int[] data) {
+		for (int i = 0; i < min.length; i++) {
+			if (min[i] > data[i]) {
+				min[i] = data[i];
+			}
+		}
 	}
 
-    }
-
-    private void assertMinMax(byte[] min, byte[] max, int[] columnGroup)
-	    throws KeyGenException {
-	KeyGenerator keyGenerator = (KeyGenerator) columnSplitter;
-
-	int columnStartIndex = 0;
-	for (int i = 0; i < columnGroup.length; i++) {
-	    int col = columnGroup[i];
-	    int[] maskByteRange = getMaskByteRange(col);
-	    int[] maskBytePosition = new int[keyGenerator.getKeySizeInBytes()];
-	    updateMaskedKeyRanges(maskBytePosition, maskByteRange);
-
-	    byte[] columnMin = new byte[maskByteRange.length];
-	    System.arraycopy(min, columnStartIndex, columnMin, 0,
-		    maskByteRange.length);
-	    byte[] columnMax = new byte[maskByteRange.length];
-	    System.arraycopy(max, columnStartIndex, columnMax, 0,
-		    maskByteRange.length);
-
-	    long[] minKeyArray = keyGenerator.getKeyArray(columnMin,
-		    maskBytePosition);
-	    long[] maxKeyArray = keyGenerator.getKeyArray(columnMax,
-		    maskBytePosition);
-	    System.out.println("calculated:(min,max) for column " + col + ":("
-		    + minKeyArray[col] + "," + maxKeyArray[col] + ")");
-	    System.out.println("correct:(min,max) for column " + col + ":("
-		    + this.min[col] + "," + this.max[col] + ")");
-	    columnStartIndex += maskByteRange.length;
-	    Assert.assertEquals(minKeyArray[col], this.min[col]);
-	    Assert.assertEquals(maxKeyArray[col], this.max[col]);
+	@Test
+	public void testRowStoreMinMax() throws KeyGenException {
+
+		DataHolder[] dataHolders = getDataHolders(segmentProperties.getColumnGroupModel().getNoOfColumnStore(),
+				mdkeyData.length);
+		for (int i = 0; i < mdkeyData.length; i++) {
+			byte[][] split = segmentProperties.getFixedLengthKeySplitter().splitKey(mdkeyData[i]);
+			for (int j = 0; j < split.length; j++) {
+				dataHolders[j].addData(split[j], i);
+			}
+
+		}
+		ColGroupBlockStorage[] rowBlockStorage = new ColGroupBlockStorage[dataHolders.length];
+		for (int i = 0; i < dataHolders.length; i++) {
+
+			rowBlockStorage[i] = new ColGroupBlockStorage(dataHolders[i]);
+		}
+		int[][] columnGroup = segmentProperties.getColumnGroups();
+		for (int i = 0; i < dataHolders.length; i++) {
+			assertMinMax(colGrpMinMax[i].getMin(), rowBlockStorage[i].getMax(),
+					columnGroup[i]);
+		}
 
 	}
 
-    }
-
-    private DataHolder[] getDataHolders(int noOfColumn, int noOfRow) {
-	DataHolder[] dataHolders = new DataHolder[noOfColumn];
-	for (int colGrp = 0; colGrp < noOfColumn; colGrp++) {
-	    if (colGrpModel.isColumnar(colGrp)) {
-		dataHolders[colGrp] = new ColumnDataHolder(noOfRow);
-	    } else {
-		dataHolders[colGrp] = new ColGroupDataHolder(this.colGrpModel,
-			this.columnSplitter.getBlockKeySize()[colGrp], noOfRow,
-			colGrpMinMax[colGrp]);
-	    }
+	private void assertMinMax(byte[] min, byte[] max, int[] columnGroup)
+			throws KeyGenException {
+
+		int columnStartIndex = 0;
+		for (int i = 0; i < columnGroup.length; i++) {
+			int col = columnGroup[i];
+			int[] maskByteRange = getMaskByteRange(col);
+			int[] maskBytePosition = new int[segmentProperties.getDimensionKeyGenerator().getKeySizeInBytes()];
+			updateMaskedKeyRanges(maskBytePosition, maskByteRange);
+
+			byte[] columnMin = new byte[maskByteRange.length];
+			System.arraycopy(min, columnStartIndex, columnMin, 0, maskByteRange.length);
+			byte[] columnMax = new byte[maskByteRange.length];
+			System.arraycopy(max, columnStartIndex, columnMax, 0, maskByteRange.length);
+
+			long[] minKeyArray = segmentProperties.getDimensionKeyGenerator().getKeyArray(columnMin, maskBytePosition);
+			long[] maxKeyArray = segmentProperties.getDimensionKeyGenerator().getKeyArray(columnMax, maskBytePosition);
+			System.out.println("calculated:(min,max) for column " + col + ":("
+					+ minKeyArray[col] + "," + maxKeyArray[col] + ")");
+			System.out.println("correct:(min,max) for column " + col + ":("
+					+ this.min[col] + "," + this.max[col] + ")");
+			columnStartIndex += maskByteRange.length;
+			Assert.assertEquals(minKeyArray[col], this.min[col]);
+			Assert.assertEquals(maxKeyArray[col], this.max[col]);
+
+		}
+
 	}
-	return dataHolders;
-    }
-
-    private int[] getMaskByteRange(int col) {
-	KeyGenerator keyGenerator = (KeyGenerator) columnSplitter;
-	Set<Integer> integers = new HashSet<>();
-	int[] range = keyGenerator.getKeyByteOffsets(col);
-	for (int j = range[0]; j <= range[1]; j++) {
-	    integers.add(j);
+
+	private DataHolder[] getDataHolders(int noOfColumn, int noOfRow) {
+		DataHolder[] dataHolders = new DataHolder[noOfColumn];
+		for (int colGrp = 0; colGrp < noOfColumn; colGrp++) {
+			if (segmentProperties.getColumnGroupModel().isColumnar(colGrp)) {
+				dataHolders[colGrp] = new ColumnDataHolder(noOfRow);
+			} else {
+				dataHolders[colGrp] = new ColGroupDataHolder(
+						segmentProperties.getFixedLengthKeySplitter().getBlockKeySize()[colGrp], noOfRow,
+						colGrpMinMax[colGrp]);
+			}
+		}
+		return dataHolders;
 	}
-	int[] byteIndexs = new int[integers.size()];
-	int j = 0;
-	for (Iterator<Integer> iterator = integers.iterator(); iterator
-		.hasNext();) {
-	    Integer integer = (Integer) iterator.next();
-	    byteIndexs[j++] = integer.intValue();
+
+	private int[] getMaskByteRange(int col) {
+		Set<Integer> integers = new HashSet<>();
+		int[] range = segmentProperties.getDimensionKeyGenerator().getKeyByteOffsets(col);
+		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 = (Integer) iterator.next();
+			byteIndexs[j++] = integer.intValue();
+		}
+		return byteIndexs;
 	}
-	return byteIndexs;
-    }
 
-    private void updateMaskedKeyRanges(int[] maskedKey, int[] maskedKeyRanges) {
-	Arrays.fill(maskedKey, -1);
-	for (int i = 0; i < maskedKeyRanges.length; i++) {
-	    maskedKey[maskedKeyRanges[i]] = i;
+	private void updateMaskedKeyRanges(int[] maskedKey, int[] maskedKeyRanges) {
+		Arrays.fill(maskedKey, -1);
+		for (int i = 0; i < maskedKeyRanges.length; i++) {
+			maskedKey[maskedKeyRanges[i]] = i;
+		}
 	}
-    }
 }


[31/50] [abbrv] incubator-carbondata git commit: [BUG] Fixed Carbon Table Path while registering as External Table to Hive (#768)

Posted by ch...@apache.org.
[BUG] Fixed Carbon Table Path while registering as External Table to Hive (#768)



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

Branch: refs/heads/master
Commit: 3718dc2fda6b45cf46e50944d23a13aa5d945f89
Parents: 5e1a67b
Author: nareshpr <pr...@gmail.com>
Authored: Tue Jun 28 03:16:41 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Tue Jun 28 03:16:41 2016 +0530

----------------------------------------------------------------------
 .../carbondata/core/carbon/path/CarbonTablePath.java  |  2 +-
 .../apache/spark/sql/CarbonDatasourceRelation.scala   | 14 ++++++++------
 .../sql/execution/command/carbonTableSchema.scala     |  5 +++--
 .../spark/sql/hive/CarbonMetastoreCatalog.scala       |  2 +-
 4 files changed, 13 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3718dc2f/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
index b764458..bf2ef57 100644
--- a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
+++ b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
@@ -47,7 +47,7 @@ public class CarbonTablePath extends Path {
   protected static final String PARTITION_PREFIX = "Part";
   protected static final String CARBON_DATA_EXT = ".carbondata";
   protected static final String DATA_PART_PREFIX = "part";
-  private static final String INDEX_FILE_EXT = ".carbonindex";
+  protected static final String INDEX_FILE_EXT = ".carbonindex";
 
   protected String tablePath;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3718dc2f/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..9f534c1 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
@@ -51,12 +51,14 @@ class CarbonSource
   override def createRelation(
       sqlContext: SQLContext,
       parameters: Map[String, String]): BaseRelation = {
-    parameters.get("path") match {
-      case Some(path) => CarbonDatasourceHadoopRelation(sqlContext, Array(path), parameters)
-      case _ =>
-        val options = new CarbonOption(parameters)
-        val tableIdentifier = options.tableIdentifier.split("""\.""").toSeq
-        CarbonDatasourceRelation(tableIdentifier, None)(sqlContext)
+    if (parameters.get("tablePath") != None) {
+      val options = new CarbonOption(parameters)
+      val tableIdentifier = options.tableIdentifier.split("""\.""").toSeq
+      CarbonDatasourceRelation(tableIdentifier, None)(sqlContext)
+    } else if (parameters.get("path") != None) {
+      CarbonDatasourceHadoopRelation(sqlContext, Array(parameters.get("path").get), parameters)
+    } else {
+      sys.error("Carbon table path not found")
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3718dc2f/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 8868367..c97a2fe 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
@@ -1260,11 +1260,12 @@ private[sql] case class CreateCube(cm: tableModel) extends RunnableCommand {
       // Add Database to catalog and persist
       val catalog = CarbonEnv.getInstance(sqlContext).carbonCatalog
       // Need to fill partitioner class when we support partition
-      val cubePath = catalog.createCubeFromThrift(tableInfo, dbName, tbName, null)(sqlContext)
+      val tablePath = catalog.createCubeFromThrift(tableInfo, dbName, tbName, null)(sqlContext)
       try {
         sqlContext.sql(
           s"""CREATE TABLE $dbName.$tbName USING org.apache.spark.sql.CarbonSource""" +
-          s""" OPTIONS (cubename "$dbName.$tbName", tablePath "$cubePath") """).collect
+          s""" OPTIONS (cubename "$dbName.$tbName", tablePath "$tablePath", path "$tablePath") """)
+              .collect
       } catch {
         case e: Exception =>
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/3718dc2f/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 56c6574..f1c8721 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
@@ -355,7 +355,7 @@ class CarbonMetastoreCatalog(hive: HiveContext, val storePath: String, client: C
     logInfo(s"Table $tableName for Database $dbName created successfully.")
     LOGGER.info("Table " + tableName + " for Database " + dbName + " created successfully.")
     updateSchemasUpdatedTime(dbName, tableName)
-    schemaMetadataPath
+    carbonTablePath.getPath
   }
 
   private def updateMetadataByWrapperTable(


[07/50] [abbrv] incubator-carbondata git commit: [Issue-578] Changing the minor compaction behavior to work on segment numbers. (#737)

Posted by ch...@apache.org.
[Issue-578] Changing the minor compaction behavior to work on segment numbers. (#737)

* Minor compaction need to be done based on the number of segments.
User will give the number of segments to be merged in levels.
* Add property carbon.compaction.level.threshold, removing carbon.compaction.minor.size

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

Branch: refs/heads/master
Commit: be46423a08effd5fb784ed7d1ff1736adf44b670
Parents: d5636db
Author: ravikiran23 <ra...@gmail.com>
Authored: Sat Jun 25 13:12:30 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Sat Jun 25 13:12:30 2016 +0530

----------------------------------------------------------------------
 .../core/constants/CarbonCommonConstants.java   |  33 +--
 .../carbondata/core/util/CarbonProperties.java  |  79 ++++--
 .../spark/merger/CarbonDataMergerUtil.java      | 247 ++++++++++++-------
 .../spark/rdd/CarbonDataRDDFactory.scala        | 123 ++++-----
 4 files changed, 305 insertions(+), 177 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be46423a/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 eb10406..cd25b88 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -881,16 +881,6 @@ public final class CarbonCommonConstants {
   public static final String INVALID_SEGMENT_ID = "-1";
 
   /**
-   * Size of Minor Compaction in MBs
-   */
-  public static final String MINOR_COMPACTION_SIZE = "carbon.minor.compaction.size";
-
-  /**
-   * By default size of minor compaction in MBs.
-   */
-  public static final String DEFAULT_MINOR_COMPACTION_SIZE = "256";
-
-  /**
    * Size of Major Compaction in MBs
    */
   public static final String MAJOR_COMPACTION_SIZE = "carbon.major.compaction.size";
@@ -990,18 +980,6 @@ public final class CarbonCommonConstants {
   public static final String SEGMENT_COMPACTED = "Compacted";
 
   /**
-   * whether to include the compacted segments again for compaction or not.
-   */
-  public static final String INCLUDE_ALREADY_COMPACTED_SEGMENTS =
-      "carbon.include.compacted.segments";
-
-  /**
-   * whether to include the compacted segments again for compaction or not. default value is false.
-   * compacted load will not be compacted again in minor compaction.
-   */
-  public static final String INCLUDE_ALREADY_COMPACTED_SEGMENTS_DEFAULT =
-      "false";
-  /**
    * property for number of core to load the blocks in driver
    */
   public static final String NUMBER_OF_CORE_TO_LOAD_DRIVER_SEGMENT =
@@ -1040,6 +1018,17 @@ public final class CarbonCommonConstants {
   public static final String FILTER_INVALID_MEMBER = " Invalid Record(s) are present "
                                                      + "while filter evaluation. ";
 
+  /**
+   * Number of unmerged segments to be merged.
+   */
+  public static final String COMPACTION_SEGMENT_LEVEL_THRESHOLD =
+      "carbon.compaction.level.threshold";
+
+  /**
+   * Default count for Number of segments to be merged in levels is 4,3
+   */
+  public static final String DEFAULT_SEGMENT_LEVEL_THRESHOLD = "4,3";
+
   private CarbonCommonConstants() {
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be46423a/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
index 60cc323..a337ac4 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
@@ -712,22 +712,6 @@ public final class CarbonProperties {
   }
 
   /**
-   * returns minor compaction size value from carbon properties or default value if it is not valid
-   *
-   * @return
-   */
-  public long getMinorCompactionSize() {
-    long compactionSize;
-    try {
-      compactionSize = Long.parseLong(getProperty(CarbonCommonConstants.MINOR_COMPACTION_SIZE,
-          CarbonCommonConstants.DEFAULT_MINOR_COMPACTION_SIZE));
-    } catch (NumberFormatException e) {
-      compactionSize = Long.parseLong(CarbonCommonConstants.DEFAULT_MINOR_COMPACTION_SIZE);
-    }
-    return compactionSize;
-  }
-
-  /**
    * returns the number of loads to be preserved.
    *
    * @return
@@ -758,4 +742,67 @@ public final class CarbonProperties {
     LOGGER.info(carbonProperties.toString());
   }
 
+  /**
+   * gettting the unmerged segment numbers to be merged.
+   * @return
+   */
+  public int[] getCompactionSegmentLevelCount() {
+    String commaSeparatedLevels;
+
+    commaSeparatedLevels = getProperty(CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD,
+        CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD);
+    int[] compactionSize = getIntArray(commaSeparatedLevels);
+
+    if(null == compactionSize){
+      compactionSize = getIntArray(CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD);
+    }
+
+    return compactionSize;
+  }
+
+  /**
+   *
+   * @param commaSeparatedLevels
+   * @return
+   */
+  private int[] getIntArray(String commaSeparatedLevels) {
+    String[] levels = commaSeparatedLevels.split(",");
+    int[] compactionSize = new int[levels.length];
+    int i = 0;
+    for (String levelSize : levels) {
+      try {
+        int size = Integer.parseInt(levelSize.trim());
+        if(validate(size,100,0,-1) < 0 ){
+          // if given size is out of boundary then take default value for all levels.
+          return null;
+        }
+        compactionSize[i++] = size;
+      }
+      catch(NumberFormatException e){
+        LOGGER.error(
+            "Given value for property" + CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD
+                + " is not proper. Taking the default value "
+                + CarbonCommonConstants.DEFAULT_SEGMENT_LEVEL_THRESHOLD);
+        return null;
+      }
+    }
+    return compactionSize;
+  }
+
+  /**
+   * Validate the restrictions
+   *
+   * @param actual
+   * @param max
+   * @param min
+   * @param defaultVal
+   * @return
+   */
+  public int validate(int actual, int max, int min, int defaultVal) {
+    if (actual <= max && actual >= min) {
+      return actual;
+    }
+    return defaultVal;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be46423a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
index f536293..7accb91 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
@@ -62,36 +62,19 @@ public final class CarbonDataMergerUtil {
 
   }
 
+  /**
+   * Returns the size of all the carbondata files present in the segment.
+   * @param carbonFile
+   * @return
+   */
   private static long getSizeOfFactFileInLoad(CarbonFile carbonFile) {
     long factSize = 0;
 
-    // check if update fact is present.
-
-    CarbonFile[] factFileUpdated = carbonFile.listFiles(new CarbonFileFilter() {
-
-      @Override public boolean accept(CarbonFile file) {
-        if (file.getName().endsWith(CarbonCommonConstants.FACT_UPDATE_EXTENSION)) {
-          return true;
-        }
-        return false;
-      }
-    });
-
-    if (factFileUpdated.length != 0) {
-      for (CarbonFile fact : factFileUpdated) {
-        factSize += fact.getSize();
-      }
-      return factSize;
-    }
-
-    // normal fact case.
+    // carbon data file case.
     CarbonFile[] factFile = carbonFile.listFiles(new CarbonFileFilter() {
 
       @Override public boolean accept(CarbonFile file) {
-        if (file.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT)) {
-          return true;
-        }
-        return false;
+        return CarbonTablePath.isCarbonDataFile(file.getName());
       }
     });
 
@@ -234,25 +217,44 @@ public final class CarbonDataMergerUtil {
       CarbonLoadModel carbonLoadModel, int partitionCount, long compactionSize,
       List<LoadMetadataDetails> segments, CompactionType compactionType) {
 
+    List sortedSegments = new ArrayList(segments);
+    // sort the segment details.
+    Collections.sort(sortedSegments, new Comparator<LoadMetadataDetails>() {
+      @Override public int compare(LoadMetadataDetails seg1, LoadMetadataDetails seg2) {
+        double seg1Id = Double.parseDouble(seg1.getLoadName());
+        double seg2Id = Double.parseDouble(seg2.getLoadName());
+        if (seg1Id - seg2Id < 0) {
+          return -1;
+        }
+        if (seg1Id - seg2Id > 0) {
+          return 1;
+        }
+        return 0;
+      }
+    });
+
     // check preserve property and preserve the configured number of latest loads.
 
     List<LoadMetadataDetails> listOfSegmentsAfterPreserve =
-        checkPreserveSegmentsPropertyReturnRemaining(segments);
+        checkPreserveSegmentsPropertyReturnRemaining(sortedSegments);
 
     // filter the segments if the compaction based on days is configured.
 
     List<LoadMetadataDetails> listOfSegmentsLoadedInSameDateInterval =
         identifySegmentsToBeMergedBasedOnLoadedDate(listOfSegmentsAfterPreserve);
-
+    List<LoadMetadataDetails> listOfSegmentsToBeMerged;
     // identify the segments to merge based on the Size of the segments across partition.
+    if (compactionType.equals(CompactionType.MAJOR_COMPACTION)) {
 
-    List<LoadMetadataDetails> listOfSegmentsBelowThresholdSize =
-        identifySegmentsToBeMergedBasedOnSize(compactionSize,
-            listOfSegmentsLoadedInSameDateInterval, carbonLoadModel, partitionCount, storeLocation,
-            compactionType);
+      listOfSegmentsToBeMerged = identifySegmentsToBeMergedBasedOnSize(compactionSize,
+          listOfSegmentsLoadedInSameDateInterval, carbonLoadModel, partitionCount, storeLocation);
+    } else {
 
+      listOfSegmentsToBeMerged =
+          identifySegmentsToBeMergedBasedOnSegCount(listOfSegmentsLoadedInSameDateInterval);
+    }
 
-    return listOfSegmentsBelowThresholdSize;
+    return listOfSegmentsToBeMerged;
   }
 
   /**
@@ -373,7 +375,7 @@ public final class CarbonDataMergerUtil {
   }
 
   /**
-   * Identify the segments to be merged based on the Size.
+   * Identify the segments to be merged based on the Size in case of Major compaction.
    *
    * @param compactionSize
    * @param listOfSegmentsAfterPreserve
@@ -384,8 +386,7 @@ public final class CarbonDataMergerUtil {
    */
   private static List<LoadMetadataDetails> identifySegmentsToBeMergedBasedOnSize(
       long compactionSize, List<LoadMetadataDetails> listOfSegmentsAfterPreserve,
-      CarbonLoadModel carbonLoadModel, int partitionCount, String storeLocation,
-      CompactionType compactionType) {
+      CarbonLoadModel carbonLoadModel, int partitionCount, String storeLocation) {
 
     List<LoadMetadataDetails> segmentsToBeMerged =
         new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
@@ -393,75 +394,157 @@ public final class CarbonDataMergerUtil {
     CarbonTableIdentifier tableIdentifier =
         carbonLoadModel.getCarbonDataLoadSchema().getCarbonTable().getCarbonTableIdentifier();
 
-    // variable to store one  segment size across partition.
-    long sizeOfOneSegmentAcrossPartition = 0;
 
     // total length
     long totalLength = 0;
 
-    String includeCompactedSegments = CarbonProperties.getInstance()
-        .getProperty(CarbonCommonConstants.INCLUDE_ALREADY_COMPACTED_SEGMENTS,
-            CarbonCommonConstants.INCLUDE_ALREADY_COMPACTED_SEGMENTS_DEFAULT);
-
     // check size of each segment , sum it up across partitions
     for (LoadMetadataDetails segment : listOfSegmentsAfterPreserve) {
 
       String segId = segment.getLoadName();
-
-      // in case of minor compaction . check the property whether to include the
-      // compacted segment or not.
-      // check if the segment is compacted or not.
-      if (CompactionType.MINOR_COMPACTION.equals(compactionType) && includeCompactedSegments
-          .equalsIgnoreCase("false") && segId.contains(".")) {
-        continue;
+      // variable to store one  segment size across partition.
+      long sizeOfOneSegmentAcrossPartition =
+          getSizeOfOneSegmentAcrossPartition(partitionCount, storeLocation, tableIdentifier, segId);
+
+      // if size of a segment is greater than the Major compaction size. then ignore it.
+      if (sizeOfOneSegmentAcrossPartition > (compactionSize * 1024 * 1024)) {
+        // if already 2 segments have been found for merging then stop scan here and merge.
+        if (segmentsToBeMerged.size() > 1) {
+          break;
+        } else { // if only one segment is found then remove the earlier one in list.
+          // reset the total length to 0.
+          segmentsToBeMerged = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+          totalLength = 0;
+          continue;
+        }
       }
 
-      // calculate size across partitions
-      for (int partition = 0; partition < partitionCount; partition++) {
-
-        String loadPath = CarbonLoaderUtil
-            .getStoreLocation(storeLocation, tableIdentifier, segId, partition + "");
-
-        CarbonFile segmentFolder =
-            FileFactory.getCarbonFile(loadPath, FileFactory.getFileType(loadPath));
-
-        long sizeOfEachSegment = getSizeOfFactFileInLoad(segmentFolder);
-
-        sizeOfOneSegmentAcrossPartition += sizeOfEachSegment;
-      }
       totalLength += sizeOfOneSegmentAcrossPartition;
-      // in case of minor compaction the size of the segments should exceed the
-      // minor compaction limit then only compaction will occur.
+
       // in case of major compaction the size doesnt matter. all the segments will be merged.
       if (totalLength < (compactionSize * 1024 * 1024)) {
         segmentsToBeMerged.add(segment);
-      }
-      // in case if minor we will merge segments only when it exceeds limit
-      // so check whether limit has been exceeded. if yes then break loop.
-      if (CompactionType.MINOR_COMPACTION.equals(compactionType)) {
-        if (totalLength > (compactionSize * 1024 * 1024)) {
-          // if size of segments exceeds then take those segments and merge.
-          // i.e if 1st segment is 200mb and 2nd segment is 100mb.
-          //  and compaction size is 256mb . we need to merge these 2 loads. so added this check.
-          segmentsToBeMerged.add(segment);
+      } else { // if already 2 segments have been found for merging then stop scan here and merge.
+        if (segmentsToBeMerged.size() > 1) {
           break;
+        } else { // if only one segment is found then remove the earlier one in list and put this.
+          // reset the total length to the current identified segment.
+          segmentsToBeMerged = new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+          segmentsToBeMerged.add(segment);
+          totalLength = sizeOfOneSegmentAcrossPartition;
         }
       }
 
-      // after all partitions
-      sizeOfOneSegmentAcrossPartition = 0;
     }
 
-    // if type is minor then we need to check the total size whether it has reached the limit of
-    // compaction size.
-    if (CompactionType.MINOR_COMPACTION.equals(compactionType)) {
-      if (totalLength < compactionSize * 1024 * 1024) {
-        // no need to do the compaction.
-        segmentsToBeMerged.removeAll(segmentsToBeMerged);
+    return segmentsToBeMerged;
+  }
+
+  /**
+   * For calculating the size of a segment across all partition.
+   * @param partitionCount
+   * @param storeLocation
+   * @param tableIdentifier
+   * @param segId
+   * @return
+   */
+  private static long getSizeOfOneSegmentAcrossPartition(int partitionCount, String storeLocation,
+      CarbonTableIdentifier tableIdentifier, String segId) {
+    long sizeOfOneSegmentAcrossPartition = 0;
+    // calculate size across partitions
+    for (int partition = 0; partition < partitionCount; partition++) {
+
+      String loadPath = CarbonLoaderUtil
+          .getStoreLocation(storeLocation, tableIdentifier, segId, partition + "");
+
+      CarbonFile segmentFolder =
+          FileFactory.getCarbonFile(loadPath, FileFactory.getFileType(loadPath));
+
+      long sizeOfEachSegment = getSizeOfFactFileInLoad(segmentFolder);
+
+      sizeOfOneSegmentAcrossPartition += sizeOfEachSegment;
+    }
+    return sizeOfOneSegmentAcrossPartition;
+  }
+
+  /**
+   * Identify the segments to be merged based on the segment count
+   *
+   * @param listOfSegmentsAfterPreserve
+   * @return
+   */
+  private static List<LoadMetadataDetails> identifySegmentsToBeMergedBasedOnSegCount(
+      List<LoadMetadataDetails> listOfSegmentsAfterPreserve) {
+
+    List<LoadMetadataDetails> mergedSegments =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    List<LoadMetadataDetails> unMergedSegments =
+        new ArrayList<>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+
+    int[] noOfSegmentLevelsCount =
+        CarbonProperties.getInstance().getCompactionSegmentLevelCount();
+
+    int level1Size = 0;
+    int level2Size = 0;
+    boolean first = true;
+
+    for(int levelCount : noOfSegmentLevelsCount){
+      if(first){
+        level1Size = levelCount;
+        first = false;
       }
+      else{
+        level2Size = levelCount;
+        break;
+        // breaking as we are doing only 2 levels
+      }
+
     }
 
-    return segmentsToBeMerged;
+    int unMergeCounter = 0;
+    int mergeCounter = 0;
+
+    // check size of each segment , sum it up across partitions
+    for (LoadMetadataDetails segment : listOfSegmentsAfterPreserve) {
+
+      String segName = segment.getLoadName();
+
+      // if a segment is already merged 2 levels then it s name will become .2
+      // need to exclude those segments from minor compaction.
+      if (segName.endsWith(".2")) {
+        continue;
+      }
+
+      // check if the segment is merged or not
+
+      if (!isMergedSegment(segName)) {
+        //if it is an unmerged segment then increment counter
+        unMergeCounter++;
+        unMergedSegments.add(segment);
+        if (unMergeCounter == (level1Size)) {
+          return unMergedSegments;
+        }
+      } else {
+        mergeCounter++;
+        mergedSegments.add(segment);
+        if (mergeCounter == (level2Size)) {
+          return mergedSegments;
+        }
+      }
+    }
+    return new ArrayList<>(0);
+  }
+
+  /**
+   * To check if the segment is merged or not.
+   * @param segName
+   * @return
+   */
+  private static boolean isMergedSegment(String segName) {
+    if(segName.contains(".")){
+      return true;
+    }
+    return false;
   }
 
   /**
@@ -550,10 +633,6 @@ public final class CarbonDataMergerUtil {
 
     long compactionSize = 0;
     switch (compactionType) {
-      case MINOR_COMPACTION:
-        compactionSize = CarbonProperties.getInstance().getMinorCompactionSize();
-        break;
-
       case MAJOR_COMPACTION:
         compactionSize = CarbonProperties.getInstance().getMajorCompactionSize();
         break;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/be46423a/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 2575f0d..e534e3d 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
@@ -301,7 +301,6 @@ object CarbonDataRDDFactory extends Logging {
       compactionType = CompactionType.MAJOR_COMPACTION
     }
     else {
-      compactionSize = CarbonDataMergerUtil.getCompactionSize(CompactionType.MINOR_COMPACTION)
       compactionType = CompactionType.MINOR_COMPACTION
     }
 
@@ -320,8 +319,6 @@ object CarbonDataRDDFactory extends Logging {
     val loadStartTime = CarbonLoaderUtil.readCurrentTime()
     carbonLoadModel.setFactTimeStamp(loadStartTime)
 
-    val executor: ExecutorService = Executors.newFixedThreadPool(1)
-
     val compactionModel = CompactionModel(compactionSize,
       compactionType,
       carbonTable,
@@ -342,7 +339,6 @@ object CarbonDataRDDFactory extends Logging {
         hdfsStoreLocation,
         kettleHomePath,
         storeLocation,
-        executor,
         compactionModel,
         lock
       )
@@ -366,14 +362,14 @@ object CarbonDataRDDFactory extends Logging {
     hdfsStoreLocation: String,
     kettleHomePath: String,
     storeLocation: String,
-    executor: ExecutorService,
     compactionModel: CompactionModel,
     compactionLock: ICarbonLock): Unit = {
+    val executor: ExecutorService = Executors.newFixedThreadPool(1)
     // update the updated table status.
     readLoadMetadataDetails(carbonLoadModel, hdfsStoreLocation)
     var segList: util.List[LoadMetadataDetails] = carbonLoadModel.getLoadMetadataDetails
 
-    val loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
+    var loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
       hdfsStoreLocation,
       carbonLoadModel,
       partitioner.partitionCount,
@@ -387,49 +383,14 @@ object CarbonDataRDDFactory extends Logging {
       new Thread {
         override def run(): Unit = {
 
-          val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
-            CarbonCommonConstants
-              .DEFAULT_COLLECTION_SIZE
-          )
-          breakable {
-            while (true) {
-
-              val loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
-                hdfsStoreLocation,
-                carbonLoadModel,
-                partitioner.partitionCount,
-                compactionModel.compactionSize,
-                segList,
-                compactionModel.compactionType
-              )
-              if (loadsToMerge.size() > 1) {
-                loadsToMerge.asScala.foreach(seg => {
-                  logger.info("load identified for merge is " + seg.getLoadName)
-                }
-                )
+          while (loadsToMerge.size() > 1) {
+
+            val futureList: util.List[Future[Void]] = new util.ArrayList[Future[Void]](
+              CarbonCommonConstants
+                .DEFAULT_COLLECTION_SIZE
+            )
+            scanSegmentsAndSubmitJob(futureList)
 
-                val future: Future[Void] = executor.submit(new CompactionCallable(hdfsStoreLocation,
-                  carbonLoadModel,
-                  partitioner,
-                  storeLocation,
-                  compactionModel.carbonTable,
-                  kettleHomePath,
-                  compactionModel.cubeCreationTime,
-                  loadsToMerge,
-                  sqlContext
-                )
-                )
-                futureList.add(future)
-                segList = CarbonDataMergerUtil
-                  .filterOutAlreadyMergedSegments(segList, loadsToMerge)
-              }
-              else {
-                executor.shutdown()
-                break
-              }
-            }
-          }
-          try {
             futureList.asScala.foreach(future => {
               try {
                 future.get
@@ -440,16 +401,71 @@ object CarbonDataRDDFactory extends Logging {
               }
             }
             )
+            // scan again and deterrmine if anything is there to merge again.
+            readLoadMetadataDetails(carbonLoadModel, hdfsStoreLocation)
+            segList = carbonLoadModel.getLoadMetadataDetails
+
+            loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
+              hdfsStoreLocation,
+              carbonLoadModel,
+              partitioner.partitionCount,
+              compactionModel.compactionSize,
+              segList,
+              compactionModel.compactionType
+            )
           }
-          finally {
-            compactionLock.unlock
-          }
+          executor.shutdown()
+          compactionLock.unlock()
         }
       }.start
     }
     else {
       compactionLock.unlock()
     }
+
+    /**
+     * This will scan all the segments and submit the loads to be merged into the executor.
+     * @param futureList
+     */
+    def scanSegmentsAndSubmitJob(futureList: util.List[Future[Void]]): Unit = {
+      breakable {
+        while (true) {
+
+          val loadsToMerge = CarbonDataMergerUtil.identifySegmentsToBeMerged(
+            hdfsStoreLocation,
+            carbonLoadModel,
+            partitioner.partitionCount,
+            compactionModel.compactionSize,
+            segList,
+            compactionModel.compactionType
+          )
+          if (loadsToMerge.size() > 1) {
+            loadsToMerge.asScala.foreach(seg => {
+              logger.info("load identified for merge is " + seg.getLoadName)
+            }
+            )
+
+            val future: Future[Void] = executor.submit(new CompactionCallable(hdfsStoreLocation,
+              carbonLoadModel,
+              partitioner,
+              storeLocation,
+              compactionModel.carbonTable,
+              kettleHomePath,
+              compactionModel.cubeCreationTime,
+              loadsToMerge,
+              sqlContext
+            )
+            )
+            futureList.add(future)
+            segList = CarbonDataMergerUtil
+              .filterOutAlreadyMergedSegments(segList, loadsToMerge)
+          }
+          else {
+            break
+          }
+        }
+      }
+    }
   }
 
   def loadCarbonData(sc: SQLContext,
@@ -473,9 +489,7 @@ object CarbonDataRDDFactory extends Logging {
           .audit("Compaction request received for table " + carbonLoadModel
             .getDatabaseName + "." + carbonLoadModel.getTableName
           )
-        val compactionSize = CarbonDataMergerUtil.getCompactionSize(CompactionType.MINOR_COMPACTION)
-
-        val executor: ExecutorService = Executors.newFixedThreadPool(1)
+        val compactionSize = 0
 
         val compactionModel = CompactionModel(compactionSize,
           CompactionType.MINOR_COMPACTION,
@@ -503,7 +517,6 @@ object CarbonDataRDDFactory extends Logging {
             hdfsStoreLocation,
             kettleHomePath,
             storeLocation,
-            executor,
             compactionModel,
             lock
           )


[46/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
index 5c3bd56,0000000..59ff84f
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
@@@ -1,92 -1,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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    ExpressionResult exprLeftRes = left.evaluate(value);
 +    ExpressionResult exprRightRes = right.evaluate(value);
 +    ExpressionResult val1 = exprLeftRes;
 +    if (exprLeftRes.isNull() || exprRightRes.isNull()) {
 +      exprLeftRes.set(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 ShortType:
 +        result = exprLeftRes.getShort() > (exprRightRes.getShort());
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java
index ddb35af,0000000..06e6ec6
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java
@@@ -1,102 -1,0 +1,102 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +              val = new ExpressionResult(val.getDataType(), expressionResVal.getShort());
 +              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/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java
index c8aa6a7,0000000..54e7dd4
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java
@@@ -1,91 -1,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.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.FilterIllegalMemberException;
 +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 {
++  public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        result = elRes.getShort() <= (erRes.getShort());
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
index 7e6580a,0000000..52ca777
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
@@@ -1,93 -1,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.FilterIllegalMemberException;
 +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 {
++  public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        result = elRes.getShort() < (erRes.getShort());
 +        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() + ')';
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java
index b252793,0000000..b04b2b1
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java
@@@ -1,57 -1,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.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.FilterIllegalMemberException;
 +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));
++      try {
++        listOfExprRes.add(expr.evaluate(value));
++      } catch (FilterIllegalMemberException e) {
++        continue;
++      }
 +    }
 +    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/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
index 4f31c07,0000000..4f9a3d1
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
@@@ -1,96 -1,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.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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +        result = val1.getShort().shortValue() != val2.getShort().shortValue();
 +        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/7f722186/core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
index caa722a,0000000..eb9e6f6
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
@@@ -1,103 -1,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.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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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 ShortType:
 +              val = new ExpressionResult(val.getDataType(), exprResVal.getShort());
 +              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/7f722186/core/src/main/java/org/carbondata/scan/expression/exception/FilterIllegalMemberException.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/exception/FilterIllegalMemberException.java
index 0000000,0000000..7130113
new file mode 100644
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/expression/exception/FilterIllegalMemberException.java
@@@ -1,0 -1,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.expression.exception;
++
++import java.util.Locale;
++
++/**
++ * FilterIllegalMemberException class representing exception which can cause while evaluating
++ * filter members needs to be gracefully handled without propagating to outer layer so that
++ * the execution should not get interrupted.
++ */
++public class FilterIllegalMemberException 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 FilterIllegalMemberException(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 FilterIllegalMemberException(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 FilterIllegalMemberException(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/7f722186/core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
index c86a38d,0000000..dbc406e
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
+++ b/core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
@@@ -1,93 -1,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.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/7f722186/core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
index b3f63bd,0000000..3a847bf
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
@@@ -1,63 -1,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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterUnsupportedException, FilterIllegalMemberException {
 +    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/7f722186/core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java
index 239af8f,0000000..dba58aa
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java
@@@ -1,58 -1,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.UnaryExpression;
++import org.carbondata.scan.expression.exception.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterIllegalMemberException, 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/7f722186/core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
index 86d6c8d,0000000..43adbf4
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
+++ b/core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
@@@ -1,60 -1,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.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.FilterIllegalMemberException;
 +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 {
++  @Override public ExpressionResult evaluate(RowIntf value)
++      throws FilterIllegalMemberException, 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/7f722186/core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java
index e5dd7b4,0000000..387b1cc
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/carbondata/scan/filter/FilterExpressionProcessor.java
@@@ -1,360 -1,0 +1,350 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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());
++        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 FilterUnsupportedException
 +   * @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)) {
++              .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
++                && 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);
-             }
++          if (condExpression.getColumnList().get(0).getCarbonColumn()
++              .hasEncoding(Encoding.DICTIONARY) && !condExpression.getColumnList().get(0)
++              .getCarbonColumn().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
++            return new ConditionalFilterResolverImpl(expression, true, true);
++          } else {
++            return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
 +          }
 +        } else {
 +          return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
 +        }
 +    }
 +    return new RowLevelFilterResolverImpl(expression, false, false, tableIdentifier);
 +  }
 +
 +}


[18/50] [abbrv] incubator-carbondata git commit: [Bug]fix exception thrown when all data is bad records (#755)

Posted by ch...@apache.org.
[Bug]fix exception thrown  when all data is bad records (#755)

1. In case all the records go to bad records in in surrogate key gen step, then also level cardinality file is getting written. Fact file writer gets initialized when level cardinality file is written and after that the process fails as there is no actual data leading to data load failure.
2. The sort key step throws null pinter exception as empty object row which has null value is put from surrogate step.

Fix:
1. Write level cardinality file only if there is at least one valid row in data.
2. If there is no valid data, then do call putRow method.

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

Branch: refs/heads/master
Commit: f16eb54f9527622249bb47a2c101c91f22c1ef5f
Parents: 5b6081d
Author: manishgupta88 <to...@gmail.com>
Authored: Mon Jun 27 10:29:24 2016 +0800
Committer: david <Qi...@users.noreply.github.com>
Committed: Mon Jun 27 10:29:24 2016 +0800

----------------------------------------------------------------------
 .../csvbased/CarbonCSVBasedSeqGenStep.java                     | 6 +++---
 1 file changed, 3 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f16eb54f/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 39335d1..d5c8e61 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
@@ -457,15 +457,15 @@ public class CarbonCSVBasedSeqGenStep extends BaseStep {
       }
 
       startReadingProcess(numberOfNodes);
-      CarbonUtil.writeLevelCardinalityFile(loadFolderLoc, meta.getTableName(),
-          getUpdatedCardinality());
       badRecordslogger.closeStreams();
       if (!meta.isAggregate()) {
         closeNormalizedHierFiles();
       }
       if (writeCounter == 0) {
-        putRow(data.getOutputRowMeta(), new Object[outSize]);
+        return processWhenRowIsNull();
       }
+      CarbonUtil.writeLevelCardinalityFile(loadFolderLoc, meta.getTableName(),
+          getUpdatedCardinality());
       LOGGER.info("Record Procerssed For table: " + meta.getTableName());
       String logMessage =
           "Summary: Carbon CSV Based Seq Gen Step : " + readCounter + ": Write: " + writeCounter;


[30/50] [abbrv] incubator-carbondata git commit: [Bug] ColumnProperty: Change visibility of variable (#764)

Posted by ch...@apache.org.
[Bug] ColumnProperty: Change visibility of variable (#764)



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

Branch: refs/heads/master
Commit: 5e1a67b9b5f2a24a8cded7e7c299ae3b9d06dea8
Parents: cdfd08c
Author: ashokblend <as...@gmail.com>
Authored: Mon Jun 27 23:16:42 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Mon Jun 27 23:16:42 2016 +0530

----------------------------------------------------------------------
 .../carbondata/core/carbon/ColumnIdentifier.java  | 18 ++++++++++++++++++
 .../core/carbon/path/CarbonTablePath.java         |  6 +++---
 .../CarbonDictionaryMetadataReaderImpl.java       |  8 ++++----
 .../core/reader/CarbonDictionaryReaderImpl.java   |  8 ++++----
 .../CarbonDictionarySortIndexReaderImpl.java      | 14 +++++++++-----
 .../core/writer/CarbonDictionaryWriterImpl.java   |  4 ++--
 .../CarbonDictionarySortIndexWriterImpl.java      |  2 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala    |  5 ++---
 .../apache/spark/sql/hive/CarbonStrategies.scala  |  4 ++--
 9 files changed, 45 insertions(+), 24 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java b/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
index 928f53b..f15aed3 100644
--- a/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
+++ b/core/src/main/java/org/carbondata/core/carbon/ColumnIdentifier.java
@@ -1,3 +1,21 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.carbon;
 
 import java.io.Serializable;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
index 538a16b..b764458 100644
--- a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
+++ b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
@@ -36,9 +36,9 @@ import org.apache.hadoop.fs.Path;
 public class CarbonTablePath extends Path {
 
   protected static final String METADATA_DIR = "Metadata";
-  private static final String DICTIONARY_EXT = ".dict";
-  private static final String DICTIONARY_META_EXT = ".dictmeta";
-  private static final String SORT_INDEX_EXT = ".sortindex";
+  protected static final String DICTIONARY_EXT = ".dict";
+  protected static final String DICTIONARY_META_EXT = ".dictmeta";
+  protected static final String SORT_INDEX_EXT = ".sortindex";
   protected static final String SCHEMA_FILE = "schema";
   protected static final String TABLE_STATUS_FILE = "tablestatus";
   protected static final String FACT_DIR = "Fact";

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
index 3b4d271..512ca8d 100644
--- a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryMetadataReaderImpl.java
@@ -40,12 +40,12 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
   /**
    * carbon table identifier
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  protected CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * HDFS store path
    */
-  private String hdfsStorePath;
+  protected String hdfsStorePath;
 
   /**
    * column identifier
@@ -55,7 +55,7 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
   /**
    * dictionary metadata file path
    */
-  private String columnDictionaryMetadataFilePath;
+  protected String columnDictionaryMetadataFilePath;
 
   /**
    * dictionary metadata thrift file reader
@@ -152,7 +152,7 @@ public class CarbonDictionaryMetadataReaderImpl implements CarbonDictionaryMetad
   /**
    * This method will form the path for dictionary metadata file for a given column
    */
-  private void initFileLocation() {
+  protected void initFileLocation() {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
     this.columnDictionaryMetadataFilePath =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
index addedf6..1313535 100644
--- a/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/CarbonDictionaryReaderImpl.java
@@ -42,12 +42,12 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   /**
    * carbon table identifier
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  protected CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * HDFS store path
    */
-  private String hdfsStorePath;
+  protected String hdfsStorePath;
 
   /**
    * column name
@@ -57,7 +57,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   /**
    * dictionary file path
    */
-  private String columnDictionaryFilePath;
+  protected String columnDictionaryFilePath;
 
   /**
    * dictionary thrift file reader
@@ -197,7 +197,7 @@ public class CarbonDictionaryReaderImpl implements CarbonDictionaryReader {
   /**
    * This method will form the path for dictionary file for a given column
    */
-  private void initFileLocation() {
+  protected void initFileLocation() {
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(this.hdfsStorePath, carbonTableIdentifier);
     this.columnDictionaryFilePath = carbonTablePath

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java b/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
index eb3f8bd..99431ed 100644
--- a/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
+++ b/core/src/main/java/org/carbondata/core/reader/sortindex/CarbonDictionarySortIndexReaderImpl.java
@@ -40,7 +40,7 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   /**
    * carbonTable Identifier holding the info of databaseName and tableName
    */
-  private CarbonTableIdentifier carbonTableIdentifier;
+  protected CarbonTableIdentifier carbonTableIdentifier;
 
   /**
    * column name
@@ -50,12 +50,12 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
   /**
    * hdfs store location
    */
-  private String carbonStorePath;
+  protected String carbonStorePath;
 
   /**
    * the path of the dictionary Sort Index file
    */
-  private String sortIndexFilePath;
+  protected String sortIndexFilePath;
 
   /**
    * Column sort info thrift instance.
@@ -143,10 +143,14 @@ public class CarbonDictionarySortIndexReaderImpl implements CarbonDictionarySort
    * @throws IOException if any I/O errors occurs
    */
   private void init() throws IOException {
+    initPath();
+    openThriftReader();
+  }
+
+  protected void initPath() {
     CarbonTablePath carbonTablePath =
-        CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
+         CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);
     this.sortIndexFilePath = carbonTablePath.getSortIndexFilePath(columnIdentifier.getColumnId());
-    openThriftReader();
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java b/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
index 22e5bbe..474b746 100644
--- a/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
+++ b/core/src/main/java/org/carbondata/core/writer/CarbonDictionaryWriterImpl.java
@@ -89,12 +89,12 @@ public class CarbonDictionaryWriterImpl implements CarbonDictionaryWriter {
   /**
    * dictionary file path
    */
-  private String dictionaryFilePath;
+  protected String dictionaryFilePath;
 
   /**
    * dictionary metadata file path
    */
-  private String dictionaryMetaFilePath;
+  protected String dictionaryMetaFilePath;
 
   /**
    * start offset of dictionary chunk  for a segment

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java b/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
index d28b176..1a9553e 100644
--- a/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
+++ b/core/src/main/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImpl.java
@@ -54,7 +54,7 @@ public class CarbonDictionarySortIndexWriterImpl implements CarbonDictionarySort
   /**
    * Path of dictionary sort index file for which the sortIndex to be written
    */
-  private String sortIndexFilePath;
+  protected String sortIndexFilePath;
   /**
    * Instance of thrift writer to write the data
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/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 69026ee..b340884 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
@@ -141,7 +141,6 @@ class CarbonSqlParser()
   protected val STARTTIME = Keyword("STARTTIME")
   protected val SEGMENTS = Keyword("SEGMENTS")
   protected val SEGMENT = Keyword("SEGMENT")
-  protected val SHARED = Keyword("SHARED")
 
   protected val doubleQuotedString = "\"([^\"]+)\"".r
   protected val singleQuotedString = "'([^']+)'".r
@@ -695,8 +694,8 @@ class CarbonSqlParser()
       dictIncludeCols =
         tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(",").map(_.trim)
       dictIncludeCols.map { distIncludeCol =>
-          if (!fields.exists(x => x.column.equalsIgnoreCase(distIncludeCol))) {
-            val errormsg = "DICTIONARY_INCLUDE column: " + distIncludeCol +
+          if (!fields.exists(x => x.column.equalsIgnoreCase(distIncludeCol.trim))) {
+            val errormsg = "DICTIONARY_INCLUDE column: " + distIncludeCol.trim +
               " does not exist in table. Please check create table statement."
             throw new MalformedCarbonCommandException(errormsg)
           }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5e1a67b9/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 412ca90..171d649 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
@@ -63,7 +63,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
   /**
    * Carbon strategies for Carbon cube scanning
    */
-  private[sql] class CarbonTableScans extends Strategy {
+  protected[sql] class CarbonTableScans extends Strategy {
 
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
       case PhysicalOperation(projectList, predicates,
@@ -269,7 +269,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
     /**
      * Create carbon scan
      */
-    private def carbonScan(projectList: Seq[NamedExpression],
+    protected def carbonScan(projectList: Seq[NamedExpression],
         predicates: Seq[Expression],
         relation: CarbonRelation,
         groupExprs: Option[Seq[Expression]],


[39/50] [abbrv] incubator-carbondata git commit: [issue-777] Filter query issue for >, <, <= than filter for timestamp(#778)

Posted by ch...@apache.org.
[issue-777] Filter query issue for >, <, <= than filter for timestamp(#778)



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

Branch: refs/heads/master
Commit: 167d52799a87b2d17cd3aa2fb846cfa0a8337e63
Parents: fe7acdc
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Thu Jun 30 04:57:03 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Thu Jun 30 04:57:03 2016 +0530

----------------------------------------------------------------------
 .../org/carbondata/core/util/CarbonUtil.java    |  41 ++---
 .../impl/AggregateQueryBlockProcessor.java      |   2 +-
 .../executer/ExcludeFilterExecuterImpl.java     |   4 +-
 .../executer/IncludeFilterExecuterImpl.java     |   4 +-
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |  11 +-
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |   6 +-
 ...velRangeLessThanEqualFilterExecuterImpl.java |  66 ++++++-
 .../RowLevelRangeLessThanFiterExecuterImpl.java |  69 ++++++--
 .../filters/measurefilter/util/FilterUtil.java  |  25 +++
 .../spark/src/test/resources/filter/emp2.csv    |   9 +
 .../src/test/resources/filter/emp2allnull.csv   |   9 +
 .../src/test/resources/filter/emp2nonull.csv    |   8 +
 .../GrtLtFilterProcessorTestCase.scala          | 176 +++++++++++++++++++
 13 files changed, 375 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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 12cbb6b..b19f8d3 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -20,27 +20,11 @@
 
 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.io.*;
 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;
-import java.util.Set;
+import java.util.*;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
@@ -866,7 +850,7 @@ public final class CarbonUtil {
   }
 
   public static int getFirstIndexUsingBinarySearch(FixedLengthDimensionDataChunk dimColumnDataChunk,
-      int low, int high, byte[] compareValue) {
+      int low, int high, byte[] compareValue, boolean matchUpLimit) {
     int cmpResult = 0;
     while (high >= low) {
       int mid = (low + high) / 2;
@@ -879,11 +863,20 @@ public final class CarbonUtil {
         high = mid - 1;
       } else {
         int currentIndex = mid;
-        while (currentIndex - 1 >= 0 && ByteUtil.UnsafeComparer.INSTANCE
-            .compareTo(dimColumnDataChunk.getCompleteDataChunk(),
-                (currentIndex - 1) * compareValue.length, compareValue.length, compareValue, 0,
-                compareValue.length) == 0) {
-          --currentIndex;
+        if(!matchUpLimit) {
+          while (currentIndex - 1 >= 0 && ByteUtil.UnsafeComparer.INSTANCE
+              .compareTo(dimColumnDataChunk.getCompleteDataChunk(),
+                  (currentIndex - 1) * compareValue.length, compareValue.length, compareValue, 0,
+                  compareValue.length) == 0) {
+            --currentIndex;
+          }
+        } else {
+          while (currentIndex + 1 <= high && ByteUtil.UnsafeComparer.INSTANCE
+              .compareTo(dimColumnDataChunk.getCompleteDataChunk(),
+                  (currentIndex + 1) * compareValue.length, compareValue.length, compareValue, 0,
+                  compareValue.length) == 0) {
+            currentIndex++;
+          }
         }
         return currentIndex;
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 260b894..3a9f04c 100644
--- 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
@@ -32,7 +32,7 @@ public class AggregateQueryBlockProcessor extends AbstractDataBlockProcessor {
   /**
    * AggregateQueryScanner constructor
    *
-   * @param blockExecutionInfos
+   * @param tableBlockExecutionInfos
    */
   public AggregateQueryBlockProcessor(BlockExecutionInfo tableBlockExecutionInfos,
       FileHolder fileReader) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 1f620c8..241e2ee 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
@@ -140,7 +140,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     for (int i = 0; i < filterValues.length; i++) {
       startKey = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
+              filterValues[i], false);
       if (startKey < 0) {
         continue;
       }
@@ -175,7 +175,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
     for (int k = 0; k < filterValues.length; k++) {
       startKey = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[k]);
+              filterValues[k], false);
       if (startKey < 0) {
         continue;
       }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 5123ce7..ae43a29 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java
@@ -130,7 +130,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
+              filterValues[i], false);
       if (start < 0) {
         continue;
       }
@@ -167,7 +167,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil.getFirstIndexUsingBinarySearch(
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-            filterValues[k]);
+            filterValues[k], false);
         if (start < 0) {
           continue;
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 0efca00..45aac01 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -117,10 +117,11 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
-      start = CarbonUtil
-          .nextGreaterValueToTarget(start, (FixedLengthDimensionDataChunk) dimensionColumnDataChunk,
-              filterValues[i], numerOfRows);
+              filterValues[i], true);
+      if (start >= 0) {
+        start = CarbonUtil.nextGreaterValueToTarget(start,
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[i], numerOfRows);
+      }
       // Logic will handle the case where the range filter member is not present in block
       // in this case the binary search will return the index from where the bit sets will be
       // set inorder to apply filters. this is greater than filter so the range will be taken
@@ -175,7 +176,7 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil.getFirstIndexUsingBinarySearch(
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-            filterValues[k]);
+            filterValues[k], true);
         start = CarbonUtil.nextGreaterValueToTarget(start,
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[k], numerOfRows);
         if (start < 0) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 935c2ce..df3c843 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -118,7 +118,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
+              filterValues[i], false);
       if (start < 0) {
         start = -(start + 1);
         if (start == numerOfRows) {
@@ -168,7 +168,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil.getFirstIndexUsingBinarySearch(
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-            filterValues[k]);
+            filterValues[k], false);
         if (start < 0) {
           start = -(start + 1);
           if (start == numerOfRows) {
@@ -177,7 +177,7 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
           // Method will compare the tentative index value after binary search, this tentative
           // index needs to be compared by the filter member if its >= filter then from that
           // index the bitset will be considered for filtering process.
-          if (ByteUtil.compare(filterValues[k],dimensionColumnDataChunk.getChunkData(start))
+          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start))
               >= 0) {
             start = start + 1;
           }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 27f7935..5319d6f 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -26,6 +26,8 @@ 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.metadata.encoder.Encoding;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.carbon.processor.BlocksChunkHolder;
@@ -33,6 +35,7 @@ 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 RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilterExecuterImpl {
   protected byte[][] filterRangeValues;
@@ -88,12 +91,23 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
 
   private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
       int numerOfRows) {
+    byte[] defaultValue = null;
+    if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+      DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+          .getDirectDictionaryGenerator(
+              dimColEvaluatorInfoList.get(0).getDimension().getDataType());
+      int key = directDictionaryGenerator.generateDirectSurrogateKey(null) + 1;
+      defaultValue = FilterUtil.getMaskKey(key, dimColEvaluatorInfoList.get(0).getDimension(),
+          this.segmentProperties.getDimensionKeyGenerator());
+    }
     if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
         && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+
       return setFilterdIndexToBitSetWithColumnIndex(
-          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows, defaultValue);
+
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
   }
 
   /**
@@ -107,17 +121,35 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows,
+      byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     int[] columnIndex = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
     int start = 0;
     int last = 0;
+    int skip = 0;
     int startIndex = 0;
     byte[][] filterValues = this.filterRangeValues;
+    //find the number of default values to skip the null value in case of direct dictionary
+    if (null != defaultValue) {
+      start = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+              defaultValue, true);
+      if (start < 0) {
+        skip = -(start + 1);
+        // end of block
+        if (skip == numerOfRows) {
+          return bitSet;
+        }
+      } else {
+        skip = start;
+      }
+      startIndex = skip;
+    }
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
+              filterValues[i], true);
       if (start < 0) {
         start = -(start + 1);
         if (start == numerOfRows) {
@@ -133,7 +165,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
         }
       }
       last = start;
-      for (int j = start; j >= 0; j--) {
+      for (int j = start; j >= skip; j--) {
         bitSet.set(columnIndex[j]);
         last--;
       }
@@ -153,20 +185,38 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
    *
    * @param dimensionColumnDataChunk
    * @param numerOfRows
+   * @param defaultValue
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
-      int numerOfRows) {
+      int numerOfRows, byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
       int start = 0;
       int last = 0;
       int startIndex = 0;
       byte[][] filterValues = this.filterRangeValues;
+      int skip = 0;
+      //find the number of default values to skip the null value in case of direct dictionary
+      if (null != defaultValue) {
+        start = CarbonUtil.getFirstIndexUsingBinarySearch(
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            defaultValue, true);
+        if (start < 0) {
+          skip = -(start + 1);
+          // end of block
+          if (skip == numerOfRows) {
+            return bitSet;
+          }
+        } else {
+          skip = start;
+        }
+        startIndex = skip;
+      }
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil.getFirstIndexUsingBinarySearch(
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-            filterValues[k]);
+            filterValues[k], true);
         if (start < 0) {
           start = -(start + 1);
           if (start == numerOfRows) {
@@ -181,7 +231,7 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
           }
         }
         last = start;
-        for (int j = start; j >= 0; j--) {
+        for (int j = start; j >= skip; j--) {
           bitSet.set(j);
           last--;
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 1786553..19213fa 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -26,6 +26,8 @@ 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.metadata.encoder.Encoding;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
+import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 import org.carbondata.core.util.ByteUtil;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.query.carbon.processor.BlocksChunkHolder;
@@ -33,6 +35,7 @@ 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 RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
   private byte[][] filterRangeValues;
@@ -88,12 +91,21 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
 
   private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
       int numerOfRows) {
+    byte[] defaultValue = null;
+    if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+      DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+          .getDirectDictionaryGenerator(
+              dimColEvaluatorInfoList.get(0).getDimension().getDataType());
+      int key = directDictionaryGenerator.generateDirectSurrogateKey(null) + 1;
+      defaultValue = FilterUtil.getMaskKey(key, dimColEvaluatorInfoList.get(0).getDimension(),
+          this.segmentProperties.getDimensionKeyGenerator());
+    }
     if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
         && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
       return setFilterdIndexToBitSetWithColumnIndex(
-          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows, defaultValue);
     }
-    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
   }
 
   /**
@@ -107,17 +119,37 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSetWithColumnIndex(
-      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows) {
+      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows,
+      byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     int[] columnIndex = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
     int start = 0;
     int last = 0;
     int startIndex = 0;
+    int skip = 0;
     byte[][] filterValues = this.filterRangeValues;
+
+    //find the number of default values to skip the null value in case of direct dictionary
+    if (null != defaultValue) {
+      start = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+              defaultValue, false);
+      if (start < 0) {
+        skip = -(start + 1);
+        // end of block
+        if (skip == numerOfRows) {
+          return bitSet;
+        }
+      } else {
+        skip = start;
+      }
+      startIndex = skip;
+    }
+
     for (int i = 0; i < filterValues.length; i++) {
       start = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-              filterValues[i]);
+              filterValues[i], false);
       // Logic will handle the case where the range filter member is not present in block
       // in this case the binary search will return the index from where the bit sets will be
       // set inorder to apply filters. this is Lesser than filter so the range will be taken
@@ -138,7 +170,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
         }
       }
       last = start;
-      for (int j = start; j >= 0; j--) {
+      for (int j = start; j >= skip; j--) {
         bitSet.set(columnIndex[j]);
         last--;
       }
@@ -161,23 +193,40 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
    * @return BitSet.
    */
   private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
-      int numerOfRows) {
+      int numerOfRows, byte[] defaultValue) {
     BitSet bitSet = new BitSet(numerOfRows);
     if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
       int start = 0;
       int last = 0;
       int startIndex = 0;
+      int skip = 0;
       byte[][] filterValues = this.filterRangeValues;
+      //find the number of default values to skip the null value in case of direct dictionary
+      if (null != defaultValue) {
+        start = CarbonUtil.getFirstIndexUsingBinarySearch(
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            defaultValue, false);
+        if (start < 0) {
+          skip = -(start + 1);
+          // end of block
+          if (skip == numerOfRows) {
+            return bitSet;
+          }
+        } else {
+          skip = start;
+        }
+        startIndex = skip;
+      }
       for (int k = 0; k < filterValues.length; k++) {
         start = CarbonUtil.getFirstIndexUsingBinarySearch(
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
-            filterValues[k]);
+            filterValues[k], false);
         start = CarbonUtil.nextLesserValueToTarget(start,
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[k]);
         if (start < 0) {
           start = -(start + 1);
-          if (start == numerOfRows) {
-            start = start - 1;
+          if (start >= numerOfRows) {
+            start = numerOfRows - 1;
           }
           // Method will compare the tentative index value after binary search, this tentative
           // index needs to be compared by the filter member if its < filter then from that
@@ -187,7 +236,7 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
           }
         }
         last = start;
-        for (int j = start; j >= 0; j--) {
+        for (int j = start; j >= skip; j--) {
           bitSet.set(j);
           last--;
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/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
index 8bae3fc..a7a78b0 100644
--- 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
@@ -605,12 +605,37 @@ public final class FilterUtil {
           LOGGER.error(e.getMessage());
         }
       }
+
     }
     return filterValuesList.toArray(new byte[filterValuesList.size()][]);
 
   }
 
   /**
+   * The method is used to get the single dictionary key's mask key
+   * @param surrogate
+   * @param carbonDimension
+   * @param blockLevelKeyGenerator
+   * @return
+   */
+  public static byte[] getMaskKey(int surrogate, CarbonDimension carbonDimension,
+      KeyGenerator blockLevelKeyGenerator) {
+
+    int[] keys = new int[blockLevelKeyGenerator.getDimCount()];
+    byte[] maskedKey = null;
+    Arrays.fill(keys, 0);
+    int[] rangesForMaskedByte =
+        getRangesForMaskedByte((carbonDimension.getKeyOrdinal()), blockLevelKeyGenerator);
+    try {
+      keys[carbonDimension.getKeyOrdinal()] = surrogate;
+      maskedKey = getMaskedKey(rangesForMaskedByte, blockLevelKeyGenerator.generateKey(keys));
+    } catch (KeyGenException e) {
+      LOGGER.error(e.getMessage());
+    }
+    return maskedKey;
+  }
+
+  /**
    * Method will return the start key based on KeyGenerator for the respective
    * filter resolved instance.
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/integration/spark/src/test/resources/filter/emp2.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/filter/emp2.csv b/integration/spark/src/test/resources/filter/emp2.csv
new file mode 100644
index 0000000..171a3c9
--- /dev/null
+++ b/integration/spark/src/test/resources/filter/emp2.csv
@@ -0,0 +1,9 @@
+empid,ename,sal,deptno,mgr,gender,dob,comm,desc
+1,abc,1233,10,2,,2014-07-01 12:07:28,1234.191,string_null
+2,bcd,1322,,3,f,2014-07-01 12:07:28,19.99,int_null
+3,cde,4322,,4,m,,16.996,date_null
+4,    ,43243,,5,m,,999.117,string_space
+5,,43242,20,6,m,2017-07-01 12:07:28,99.999,string_null
+6,ijk,,20,6,m,2017-07-01 12:07:28,50089,double_null
+7,pqr,2422,20,6,m,2017-07-01 12:07:28,32.339,decimal_null
+8

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/integration/spark/src/test/resources/filter/emp2allnull.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/filter/emp2allnull.csv b/integration/spark/src/test/resources/filter/emp2allnull.csv
new file mode 100644
index 0000000..9a1962c
--- /dev/null
+++ b/integration/spark/src/test/resources/filter/emp2allnull.csv
@@ -0,0 +1,9 @@
+empid,ename,sal,deptno,mgr,gender,dob,comm,desc
+1,abc,1233,10,2,,,1234.191,string_null
+2,bcd,1322,,3,f,,19.99,int_null
+3,cde,4322,,4,m,,16.996,date_null
+4,    ,43243,,5,m,,999.117,string_space
+5,,43242,20,6,m,,99.999,string_null
+6,ijk,,20,6,m,,50089,double_null
+7,pqr,2422,20,6,m,,32.339,decimal_null
+8

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/integration/spark/src/test/resources/filter/emp2nonull.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/filter/emp2nonull.csv b/integration/spark/src/test/resources/filter/emp2nonull.csv
new file mode 100644
index 0000000..db3bd2c
--- /dev/null
+++ b/integration/spark/src/test/resources/filter/emp2nonull.csv
@@ -0,0 +1,8 @@
+empid,ename,sal,deptno,mgr,gender,dob,comm,desc
+1,abc,1233,10,2,,2014-07-01 12:07:28,1234.191,string_null
+2,bcd,1322,,3,f,2014-07-01 12:07:28,19.99,int_null
+3,cde,4322,,4,m,2014-07-01 12:07:28,16.996,date_null
+4,    ,43243,,5,m,2014-07-01 12:07:28,999.117,string_space
+5,,43242,20,6,m,2017-07-01 12:07:28,99.999,string_null
+6,ijk,,20,6,m,2017-07-01 12:07:28,50089,double_null
+7,pqr,2422,20,6,m,2017-07-01 12:07:28,32.339,decimal_null

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/167d5279/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
new file mode 100644
index 0000000..8c51106
--- /dev/null
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/GrtLtFilterProcessorTestCase.scala
@@ -0,0 +1,176 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.filterexpr
+
+import java.io.File
+
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.common.util.CarbonHiveContext._
+import org.apache.spark.sql.common.util.QueryTest
+import org.carbondata.core.constants.CarbonCommonConstants
+import org.carbondata.core.util.CarbonProperties
+import org.scalatest.BeforeAndAfterAll
+
+/**
+  * Test Class for filter expression query on String datatypes
+  *
+  */
+class GrtLtFilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
+
+  override def beforeAll {
+    sql("drop table if exists a12")
+    sql("drop table if exists a12_allnull")
+    sql("drop table if exists a12_no_null")
+
+    sql(
+      "create table a12(empid String,ename String,sal double,deptno int,mgr string,gender string," +
+        "dob timestamp,comm decimal(4,2),desc string) stored by 'org.apache.carbondata.format'"
+    )
+    sql(
+      "create table a12_allnull(empid String,ename String,sal double,deptno int,mgr string,gender" +
+        " string," +
+        "dob timestamp,comm decimal(4,2),desc string) stored by 'org.apache.carbondata.format'"
+    )
+    sql(
+      "create table a12_no_null(empid String,ename String,sal double,deptno int,mgr string,gender" +
+        " string," +
+        "dob timestamp,comm decimal(4,2),desc string) stored by 'org.apache.carbondata.format'"
+    )
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
+    val basePath = new File(this.getClass.getResource("/").getPath + "/../../")
+      .getCanonicalPath
+    var testData = basePath + "/src/test/resources/filter/emp2.csv"
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$testData' into table a12 OPTIONS('DELIMITER'=',',
+         'QUOTECHAR'='"','FILEHEADER'='empid,ename,sal,deptno,mgr,gender,dob,comm,desc')"""
+        .stripMargin
+    )
+    testData = basePath + "/src/test/resources/filter/emp2allnull.csv"
+
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$testData' into table a12_allnull OPTIONS('DELIMITER'=',',
+         'QUOTECHAR'='"','FILEHEADER'='empid,ename,sal,deptno,mgr,gender,dob,comm,desc')"""
+        .stripMargin
+    )
+    testData = basePath + "/src/test/resources/filter/emp2nonull.csv"
+
+    sql(
+      s"""LOAD DATA LOCAL INPATH '$testData' into table a12_no_null OPTIONS('DELIMITER'=',',
+         'QUOTECHAR'='"')"""
+        .stripMargin
+    )
+  }
+  //mixed value test
+  test("Less Than Filter") {
+    checkAnswer(
+      sql("select count(empid) from a12 where dob < '2014-07-01 12:07:28'"),
+      Seq(Row(0))
+    )
+  }
+
+  test("Les Than equal Filter") {
+    checkAnswer(
+      sql("select count (empid) from a12 where dob <= '2014-07-01 12:07:28'"),
+      Seq(Row(2))
+    )
+  }
+
+  test("Greater Than Filter") {
+    checkAnswer(
+      sql("select count (empid) from a12 where dob > '2014-07-01 12:07:28'"),
+      Seq(Row(3))
+    )
+  }
+
+  test("Greater Than equal to Filter") {
+    sql("select count (empid) from a12 where dob >= '2014-07-01 12:07:28'").show()
+    checkAnswer(
+      sql("select count (empid) from a12 where dob >= '2014-07-01 12:07:28'"),
+      Seq(Row(5))
+    )
+  }
+  //all null test cases
+
+  test("Less Than Filter all null") {
+    checkAnswer(
+      sql("select count(empid) from a12_allnull where dob < '2014-07-01 12:07:28'"),
+      Seq(Row(0))
+    )
+  }
+
+  test("Les Than equal Filter all null") {
+    checkAnswer(
+      sql("select count (empid) from a12_allnull where dob <= '2014-07-01 12:07:28'"),
+      Seq(Row(0))
+    )
+  }
+
+  test("Greater Than Filter all null") {
+    checkAnswer(
+      sql("select count (empid) from a12_allnull where dob > '2014-07-01 12:07:28'"),
+      Seq(Row(0))
+    )
+  }
+
+  test("Greater Than equal to Filter all null") {
+    checkAnswer(
+      sql("select count (empid) from a12_allnull where dob >= '2014-07-01 12:07:28'"),
+      Seq(Row(0))
+    )
+  }
+
+  //no null test cases
+
+  test("Less Than Filter no null") {
+    checkAnswer(
+      sql("select count(empid) from a12_no_null where dob < '2014-07-01 12:07:28'"),
+      Seq(Row(0))
+    )
+  }
+
+  test("Les Than equal Filter no null") {
+    sql("select empid from a12_no_null where dob <= '2014-07-01 12:07:28'").show()
+    checkAnswer(
+      sql("select count(empid) from a12_no_null where dob <= '2014-07-01 12:07:28'"),
+      Seq(Row(4))
+    )
+  }
+
+  test("Greater Than Filter no null") {
+    checkAnswer(
+      sql("select count (empid) from a12_no_null where dob > '2014-07-01 12:07:28'"),
+      Seq(Row(3))
+    )
+  }
+
+  test("Greater Than equal to Filter no null") {
+    checkAnswer(
+      sql("select count (empid) from a12_no_null where dob >= '2014-07-01 12:07:28'"),
+      Seq(Row(7))
+    )
+  }
+
+  override def afterAll {
+    sql("drop table a12")
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+  }
+}



[22/50] [abbrv] incubator-carbondata git commit: [Issue-643] Column Property addition, extract interface for dictionary (#641)

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/core/src/test/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java b/core/src/test/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
index 17976a6..b814fa8 100644
--- a/core/src/test/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
+++ b/core/src/test/java/org/carbondata/core/writer/sortindex/CarbonDictionarySortIndexWriterImplTest.java
@@ -24,6 +24,7 @@ import java.util.List;
 import java.util.UUID;
 
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.datastorage.store.filesystem.CarbonFile;
 import org.carbondata.core.datastorage.store.impl.FileFactory;
 import org.carbondata.core.reader.sortindex.CarbonDictionarySortIndexReader;
@@ -59,8 +60,10 @@ public class CarbonDictionarySortIndexWriterImplTest {
   @Test public void write() throws Exception {
     String storePath = hdfsStorePath;
     CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("testSchema", "carbon", UUID.randomUUID().toString());
+    ColumnIdentifier columnIdentifier = new ColumnIdentifier("Name", null, null);
+
     CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, "Name", storePath);
+        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, columnIdentifier, storePath);
     List<int[]> indexList = prepareExpectedData();
     List<Integer> sortIndex = Arrays.asList(ArrayUtils.toObject(indexList.get(0)));
     List<Integer> invertedSortIndex = Arrays.asList(ArrayUtils.toObject(indexList.get(1)));
@@ -68,7 +71,7 @@ public class CarbonDictionarySortIndexWriterImplTest {
     dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
     dictionarySortIndexWriter.close();
     CarbonDictionarySortIndexReader carbonDictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, "Name", storePath);
+        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier, storePath);
     List<Integer> actualSortIndex = carbonDictionarySortIndexReader.readSortIndex();
     List<Integer> actualInvertedSortIndex = carbonDictionarySortIndexReader.readInvertedSortIndex();
     for (int i = 0; i < actualSortIndex.size(); i++) {
@@ -84,15 +87,17 @@ public class CarbonDictionarySortIndexWriterImplTest {
   @Test public void writingEmptyValue() throws Exception {
     String storePath = hdfsStorePath;
     CarbonTableIdentifier carbonTableIdentifier = new CarbonTableIdentifier("testSchema", "carbon", UUID.randomUUID().toString());
+    ColumnIdentifier columnIdentifier = new ColumnIdentifier("Name", null, null);
+
     CarbonDictionarySortIndexWriter dictionarySortIndexWriter =
-        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, "Name", storePath);
+        new CarbonDictionarySortIndexWriterImpl(carbonTableIdentifier, columnIdentifier, storePath);
     List<Integer> sortIndex = new ArrayList<>();
     List<Integer> invertedSortIndex = new ArrayList<>();
     dictionarySortIndexWriter.writeSortIndex(sortIndex);
     dictionarySortIndexWriter.writeInvertedSortIndex(invertedSortIndex);
     dictionarySortIndexWriter.close();
     CarbonDictionarySortIndexReader carbonDictionarySortIndexReader =
-        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, "Name", storePath);
+        new CarbonDictionarySortIndexReaderImpl(carbonTableIdentifier, columnIdentifier, storePath);
     List<Integer> actualSortIndex = carbonDictionarySortIndexReader.readSortIndex();
     List<Integer> actualInvertedSortIndex = carbonDictionarySortIndexReader.readInvertedSortIndex();
     for (int i = 0; i < actualSortIndex.size(); i++) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 2d37bda..deef2cc 100644
--- a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
@@ -20,7 +20,7 @@ package org.carbondata.examples
 import org.apache.spark.sql.{CarbonContext, CarbonEnv, CarbonRelation}
 
 import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
-import org.carbondata.core.carbon.CarbonTableIdentifier
+import org.carbondata.core.carbon.{CarbonTableIdentifier, ColumnIdentifier}
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 import org.carbondata.core.carbon.path.CarbonStorePath
 import org.carbondata.examples.util.InitForExamples
@@ -76,7 +76,7 @@ object GenerateDictionaryExample {
       println(s"dictionary of dimension: ${dimension.getColName}")
       println(s"Key\t\t\tValue")
       val columnIdentifier = new DictionaryColumnUniqueIdentifier(carbonTableIdentifier,
-        dimension.getColumnId, dimension.getDataType)
+        dimension.getColumnIdentifier, dimension.getDataType)
       val dict = CarbonLoaderUtil.getDictionary(columnIdentifier, carbonContext.storePath)
       var index: Int = 1
       var distinctValue = dict.getDictionaryValueForKey(index)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------
diff --git a/format/src/main/thrift/schema.thrift b/format/src/main/thrift/schema.thrift
index 8e12361..690e37b 100644
--- a/format/src/main/thrift/schema.thrift
+++ b/format/src/main/thrift/schema.thrift
@@ -85,10 +85,19 @@ struct ColumnSchema{
 	11: optional string aggregate_function;
 
 	12: optional binary default_value;
-	/**
+	
+	13: optional map<string,string> columnProperties
+	
+    /**
 	* To specify the visibily of the column by default its false
 	*/
-	13: optional bool invisible;
+	14: optional bool invisible;
+
+	/**
+	 * column reference id
+	 */
+	15: optional string columnReferenceId;	
+	
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/hadoop/src/main/java/org/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java b/hadoop/src/main/java/org/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
index ba1aafc..bea8289 100644
--- a/hadoop/src/main/java/org/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
+++ b/hadoop/src/main/java/org/carbondata/hadoop/readsupport/impl/AbstractDictionaryDecodedReadSupport.java
@@ -47,8 +47,8 @@ public abstract class AbstractDictionaryDecodedReadSupport<T> implements CarbonR
         try {
           dataTypes[i] = carbonColumns[i].getDataType();
           dictionaries[i] = forwardDictionaryCache.get(new DictionaryColumnUniqueIdentifier(
-              absoluteTableIdentifier.getCarbonTableIdentifier(), carbonColumns[i].getColumnId(),
-              dataTypes[i]));
+              absoluteTableIdentifier.getCarbonTableIdentifier(),
+              carbonColumns[i].getColumnIdentifier(), dataTypes[i]));
         } catch (CarbonUtilException e) {
           throw new RuntimeException(e);
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 556fc8d..ad21250 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
@@ -41,6 +41,7 @@ import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.CarbonDataLoadSchema;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.converter.SchemaConverter;
 import org.carbondata.core.carbon.metadata.converter.ThriftWrapperSchemaConverterImpl;
@@ -289,9 +290,10 @@ public class StoreCreator {
     Cache dictCache = CacheProvider.getInstance()
         .createCache(CacheType.REVERSE_DICTIONARY, absoluteTableIdentifier.getStorePath());
     for (int i = 0; i < set.length; i++) {
+      ColumnIdentifier columnIdentifier = new ColumnIdentifier(dims.get(i).getColumnId(), null, null);
       CarbonDictionaryWriter writer =
           new CarbonDictionaryWriterImpl(absoluteTableIdentifier.getStorePath(),
-              absoluteTableIdentifier.getCarbonTableIdentifier(), dims.get(i).getColumnId());
+              absoluteTableIdentifier.getCarbonTableIdentifier(), columnIdentifier);
       for (String value : set[i]) {
         writer.write(value);
       }
@@ -299,14 +301,14 @@ public class StoreCreator {
 
       Dictionary dict = (Dictionary) dictCache.get(
           new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier.getCarbonTableIdentifier(),
-              dims.get(i).getColumnId(), dims.get(i).getDataType()));
+        		  columnIdentifier, dims.get(i).getDataType()));
       CarbonDictionarySortInfoPreparator preparator =
           new CarbonDictionarySortInfoPreparator();
       CarbonDictionarySortInfo dictionarySortInfo =
           preparator.getDictionarySortInfo(dict, dims.get(i).getDataType());
       CarbonDictionarySortIndexWriter carbonDictionaryWriter =
           new CarbonDictionarySortIndexWriterImpl(
-              absoluteTableIdentifier.getCarbonTableIdentifier(), dims.get(i).getColumnId(),
+              absoluteTableIdentifier.getCarbonTableIdentifier(), columnIdentifier,
               absoluteTableIdentifier.getStorePath());
       try {
         carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex());

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 c5eb971..220d1b7 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
@@ -32,6 +32,7 @@ import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier;
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.CarbonDataLoadSchema;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
+import org.carbondata.core.carbon.ColumnIdentifier;
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.carbondata.core.carbon.metadata.CarbonMetadata;
 import org.carbondata.core.carbon.metadata.datatype.DataType;
@@ -979,7 +980,7 @@ public final class CarbonLoaderUtil {
   }
 
   public static Dictionary getDictionary(CarbonTableIdentifier tableIdentifier,
-      String columnIdentifier, String carbonStorePath, DataType dataType)
+      ColumnIdentifier columnIdentifier, String carbonStorePath, DataType dataType)
       throws CarbonUtilException {
     return getDictionary(
         new DictionaryColumnUniqueIdentifier(tableIdentifier, columnIdentifier, dataType),

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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..ce43c4f 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
@@ -29,7 +29,7 @@ import org.apache.spark.unsafe.types.UTF8String
 
 import org.carbondata.core.cache.{Cache, CacheProvider, CacheType}
 import org.carbondata.core.cache.dictionary.{Dictionary, DictionaryColumnUniqueIdentifier}
-import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifier}
+import org.carbondata.core.carbon.{AbsoluteTableIdentifier, CarbonTableIdentifier, ColumnIdentifier}
 import org.carbondata.core.carbon.metadata.datatype.DataType
 import org.carbondata.core.carbon.metadata.encoder.Encoding
 import org.carbondata.query.carbon.util.DataTypeUtil
@@ -102,7 +102,7 @@ case class CarbonDictionaryDecoder(
 
   val getDictionaryColumnIds = {
     val attributes = child.output
-    val dictIds: Array[(String, String, DataType)] = attributes.map { a =>
+    val dictIds: Array[(String, ColumnIdentifier, DataType)] = attributes.map { a =>
       val attr = aliasMap.getOrElse(a, a)
       val relation = relations.find(p => p.contains(attr))
       if(relation.isDefined) {
@@ -113,7 +113,8 @@ case class CarbonDictionaryDecoder(
             carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
             !carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
             canBeDecoded(attr)) {
-          (carbonTable.getFactTableName, carbonDimension.getColumnId, carbonDimension.getDataType)
+          (carbonTable.getFactTableName, carbonDimension.getColumnIdentifier,
+              carbonDimension.getDataType)
         } else {
           (null, null, null)
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 98fc334..e766aaa 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
@@ -18,6 +18,7 @@
 package org.apache.spark.sql
 
 import java.nio.charset.Charset
+import java.util
 import java.util.regex.{Matcher, Pattern}
 
 import scala.collection.JavaConverters._
@@ -37,6 +38,7 @@ 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
@@ -139,6 +141,7 @@ class CarbonSqlParser()
   protected val STARTTIME = Keyword("STARTTIME")
   protected val SEGMENTS = Keyword("SEGMENTS")
   protected val SEGMENT = Keyword("SEGMENT")
+  protected val SHARED = Keyword("SHARED")
 
   protected val doubleQuotedString = "\"([^\"]+)\"".r
   protected val singleQuotedString = "'([^']+)'".r
@@ -176,7 +179,8 @@ class CarbonSqlParser()
   }
 
   override protected lazy val start: Parser[LogicalPlan] =
-    createCube | showCreateCube | loadManagement | createAggregateTable | describeTable |
+     createCube | showCreateCube | loadManagement | createAggregateTable |
+      describeTable |
       showCube | showLoads | alterCube | showAllCubes | alterTable | createTable
 
   protected lazy val loadManagement: Parser[LogicalPlan] = loadData | dropCubeOrTable |
@@ -467,10 +471,14 @@ class CarbonSqlParser()
           sys.error("Not a carbon format request")
         }
 
-        // prepare table model of the collected tokens
-        val tableModel: tableModel = prepareTableModel(ifNotExistPresent, dbName, tableName, fields,
-          partitionCols,
-          tableProperties)
+      // validate tblProperties
+      if (!CommonUtil.validateTblProperties(tableProperties, fields)) {
+        throw new MalformedCarbonCommandException("Invalid table properties")
+      }
+      // prepare table model of the collected tokens
+      val tableModel: tableModel = prepareTableModel(ifNotExistPresent, dbName, tableName, fields,
+        partitionCols,
+        tableProperties)
 
         // get logical plan.
         CreateCube(tableModel)
@@ -538,6 +546,8 @@ class CarbonSqlParser()
       fields, tableProperties)
     val msrs: Seq[Field] = extractMsrColsFromFields(fields, tableProperties)
 
+    // column properties
+    val colProps = extractColumnProperties(fields, tableProperties)
     // get column groups configuration from table properties.
     val groupCols: Seq[String] = updateColumnGroupsInField(tableProperties,
         noDictionaryDims, msrs, dims)
@@ -548,7 +558,7 @@ class CarbonSqlParser()
       dbName.getOrElse("default"), dbName, tableName,
       reorderDimensions(dims.map(f => normalizeType(f)).map(f => addParent(f))),
       msrs.map(f => normalizeType(f)), "", null, "",
-      None, Seq(), null, Option(noDictionaryDims), null, partitioner, groupCols)
+      None, Seq(), null, Option(noDictionaryDims), null, partitioner, groupCols, Some(colProps))
   }
 
   /**
@@ -562,10 +572,10 @@ class CarbonSqlParser()
       noDictionaryDims: Seq[String],
       msrs: Seq[Field],
       dims: Seq[Field]): Seq[String] = {
-    if (None != tableProperties.get("COLUMN_GROUPS")) {
+    if (None != tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS)) {
 
       var splittedColGrps: Seq[String] = Seq[String]()
-      val nonSplitCols: String = tableProperties.get("COLUMN_GROUPS").get
+      val nonSplitCols: String = tableProperties.get(CarbonCommonConstants.COLUMN_GROUPS).get
 
       // row groups will be specified in table properties like -> "(col1,col2),(col3,col4)"
       // here first splitting the value by () . so that the above will be splitted into 2 strings.
@@ -637,13 +647,13 @@ class CarbonSqlParser()
     var partitionClass: String = ""
     var partitionCount: Int = 1
     var partitionColNames: Array[String] = Array[String]()
-    if (None != tableProperties.get("PARTITIONCLASS")) {
-      partitionClass = tableProperties.get("PARTITIONCLASS").get
+    if (None != tableProperties.get(CarbonCommonConstants.PARTITIONCLASS)) {
+      partitionClass = tableProperties.get(CarbonCommonConstants.PARTITIONCLASS).get
     }
 
-    if (None != tableProperties.get("PARTITIONCOUNT")) {
+    if (None != tableProperties.get(CarbonCommonConstants.PARTITIONCOUNT)) {
       try {
-        partitionCount = tableProperties.get("PARTITIONCOUNT").get.toInt
+        partitionCount = tableProperties.get(CarbonCommonConstants.PARTITIONCOUNT).get.toInt
       } catch {
         case e: Exception => // no need to do anything.
       }
@@ -661,6 +671,54 @@ class CarbonSqlParser()
     None
   }
 
+  protected def extractColumnProperties(fields: Seq[Field], tableProperties: Map[String, String]):
+  util.Map[String, util.List[ColumnProperty]] = {
+    val colPropMap = new util.HashMap[String, util.List[ColumnProperty]]()
+    fields.foreach { field =>
+      if (field.children.isDefined && field.children.get != null) {
+        fillAllChildrenColumnProperty(field.column, field.children, tableProperties, colPropMap)
+      } else {
+        fillColumnProperty(None, field.column, tableProperties, colPropMap)
+      }
+    }
+    colPropMap
+  }
+
+  protected def fillAllChildrenColumnProperty(parent: String, fieldChildren: Option[List[Field]],
+    tableProperties: Map[String, String],
+    colPropMap: util.HashMap[String, util.List[ColumnProperty]]) {
+    fieldChildren.foreach(fields => {
+      fields.foreach(field => {
+        fillColumnProperty(Some(parent), field.column, tableProperties, colPropMap)
+      }
+      )
+    }
+    )
+  }
+
+  protected def fillColumnProperty(parentColumnName: Option[String],
+    columnName: String,
+    tableProperties: Map[String, String],
+    colPropMap: util.HashMap[String, util.List[ColumnProperty]]) {
+    val (tblPropKey, colProKey) = getKey(parentColumnName, columnName)
+    val colProps = CommonUtil.getColumnProperties(tblPropKey, tableProperties)
+    if (None != colProps) {
+      colPropMap.put(colProKey, colProps.get)
+    }
+  }
+
+  def getKey(parentColumnName: Option[String],
+    columnName: String): (String, String) = {
+    if (None != parentColumnName) {
+      if (columnName == "val") {
+        (parentColumnName.get, parentColumnName.get + "." + columnName)
+      } else {
+        (parentColumnName.get + "." + columnName, parentColumnName.get + "." + columnName)
+      }
+    } else {
+      (columnName, columnName)
+    }
+  }
   /**
    * This will extract the Dimensions and NoDictionary Dimensions fields.
    * By default all string cols are dimensions.
@@ -678,8 +736,9 @@ class CarbonSqlParser()
     var dictIncludeCols: Seq[String] = Seq[String]()
 
     // All excluded cols should be there in create table cols
-    if (tableProperties.get("DICTIONARY_EXCLUDE").isDefined) {
-      dictExcludeCols = tableProperties.get("DICTIONARY_EXCLUDE").get.split(',').map(_.trim)
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).isDefined) {
+      dictExcludeCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
       dictExcludeCols
         .map { dictExcludeCol =>
           if (!fields.exists(x => x.column.equalsIgnoreCase(dictExcludeCol))) {
@@ -702,8 +761,9 @@ class CarbonSqlParser()
         }
     }
     // All included cols should be there in create table cols
-    if (tableProperties.get("DICTIONARY_INCLUDE").isDefined) {
-      dictIncludeCols = tableProperties.get("DICTIONARY_INCLUDE").get.split(",").map(_.trim)
+    if (tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).isDefined) {
+      dictIncludeCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(",").map(_.trim)
       dictIncludeCols.map { distIncludeCol =>
           if (!fields.exists(x => x.column.equalsIgnoreCase(distIncludeCol))) {
             val errormsg = "DICTIONARY_INCLUDE column: " + distIncludeCol +
@@ -800,13 +860,15 @@ class CarbonSqlParser()
     var dictExcludedCols: Array[String] = Array[String]()
 
     // get all included cols
-    if (None != tableProperties.get("DICTIONARY_INCLUDE")) {
-      dictIncludedCols = tableProperties.get("DICTIONARY_INCLUDE").get.split(',').map(_.trim)
+    if (None != tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE)) {
+      dictIncludedCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_INCLUDE).get.split(',').map(_.trim)
     }
 
     // get all excluded cols
-    if (None != tableProperties.get("DICTIONARY_EXCLUDE")) {
-      dictExcludedCols = tableProperties.get("DICTIONARY_EXCLUDE").get.split(',').map(_.trim)
+    if (None != tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE)) {
+      dictExcludedCols =
+        tableProperties.get(CarbonCommonConstants.DICTIONARY_EXCLUDE).get.split(',').map(_.trim)
     }
 
     // by default consider all non string cols as msrs. consider all include/ exclude cols as dims
@@ -887,8 +949,8 @@ class CarbonSqlParser()
   }
 
   protected def unquoteString(str: String) = str match {
-    case singleQuotedString(s) => s
-    case doubleQuotedString(s) => s
+    case singleQuotedString(s) => s.toLowerCase()
+    case doubleQuotedString(s) => s.toLowerCase()
     case other => other
   }
 
@@ -1259,7 +1321,6 @@ class CarbonSqlParser()
           new DescribeCommand(UnresolvedRelation(tblIdentifier, None), ef.isDefined)
         }
     }
-
   private def normalizeType(field: Field): Field = {
     field.dataType.getOrElse("NIL") match {
       case "string" => Field(field.column, Some("String"), field.name, Some(null), field.parent,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 a65b7b7..8868367 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
@@ -37,7 +37,9 @@ import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.types.TimestampType
 import org.apache.spark.util.FileUtils
+import org.codehaus.jackson.map.ObjectMapper
 
+import org.carbondata.common.factory.CarbonCommonFactory
 import org.carbondata.common.logging.LogServiceFactory
 import org.carbondata.core.carbon.CarbonDataLoadSchema
 import org.carbondata.core.carbon.metadata.CarbonMetadata
@@ -57,7 +59,8 @@ import org.carbondata.spark.exception.MalformedCarbonCommandException
 import org.carbondata.spark.load._
 import org.carbondata.spark.partition.api.impl.QueryPartitionHelper
 import org.carbondata.spark.rdd.CarbonDataRDDFactory
-import org.carbondata.spark.util.{CarbonScalaUtil, GlobalDictionaryUtil}
+import org.carbondata.spark.util.{CarbonScalaUtil, CommonUtil, GlobalDictionaryUtil}
+import org.carbondata.spark.CarbonSparkFactory
 
 
 case class tableModel(
@@ -76,7 +79,8 @@ case class tableModel(
     highcardinalitydims: Option[Seq[String]],
     aggregation: Seq[Aggregation],
     partitioner: Option[Partitioner],
-    columnGroups: Seq[String])
+    columnGroups: Seq[String],
+    colProps: Option[util.Map[String, util.List[ColumnProperty]]] = None)
 
 case class Field(column: String, dataType: Option[String], name: Option[String],
     children: Option[List[Field]], parent: String = null,
@@ -93,6 +97,8 @@ case class FieldMapping(levelName: String, columnName: String)
 
 case class HierarchyMapping(hierName: String, hierType: String, levels: Seq[String])
 
+case class ColumnProperty(key: String, value: String)
+
 case class ComplexField(complexType: String, primitiveField: Option[Field],
     complexField: Option[ComplexField])
 
@@ -187,9 +193,26 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
     val columnSchema = new ColumnSchema()
     columnSchema.setDataType(dataType)
     columnSchema.setColumnName(colName)
-    columnSchema.setColumnUniqueId(UUID.randomUUID().toString)
-    columnSchema.setColumnar(isCol)
+    val highCardinalityDims = cm.highcardinalitydims.getOrElse(Seq())
+    if (highCardinalityDims.contains(colName)) {
+      encoders.remove(encoders.remove(Encoding.DICTIONARY))
+    }
+    if (dataType == DataType.TIMESTAMP) {
+      encoders.add(Encoding.DIRECT_DICTIONARY)
+    }
+    var colPropMap = new java.util.HashMap[String, String]()
+    if (None != cm.colProps && null != cm.colProps.get.get(colName)) {
+      val colProps = cm.colProps.get.get(colName)
+      colProps.asScala.foreach { x => colPropMap.put(x.key, x.value) }
+    }
+    columnSchema.setColumnProperties(colPropMap)
     columnSchema.setEncodingList(encoders)
+    val colUniqueIdGenerator = CarbonCommonFactory.getColumnUniqueIdGenerator
+    val columnUniqueId = colUniqueIdGenerator.generateUniqueId(cm.schemaName,
+      columnSchema)
+    columnSchema.setColumnUniqueId(columnUniqueId)
+    columnSchema.setColumnReferenceId(columnUniqueId)
+    columnSchema.setColumnar(isCol)
     columnSchema.setDimensionColumn(isDimensionCol)
     columnSchema.setColumnGroup(colGroup)
     columnSchema.setPrecision(precision)
@@ -258,15 +281,6 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
 
     updateColumnGroupsInFields(cm.columnGroups, allColumns)
 
-    for (column <- allColumns) {
-      if (highCardinalityDims.contains(column.getColumnName)) {
-        column.getEncodingList.remove(Encoding.DICTIONARY)
-      }
-      if (column.getDataType == DataType.TIMESTAMP) {
-        column.getEncodingList.add(Encoding.DIRECT_DICTIONARY)
-      }
-    }
-
     var newOrderedDims = scala.collection.mutable.ListBuffer[ColumnSchema]()
     val complexDims = scala.collection.mutable.ListBuffer[ColumnSchema]()
     val measures = scala.collection.mutable.ListBuffer[ColumnSchema]()
@@ -301,7 +315,8 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
       measures += measureColumn
       allColumns = allColumns ++ measures
     }
-
+    val columnValidator = CarbonSparkFactory.getCarbonColumnValidator()
+    columnValidator.validateColumns(allColumns)
     newOrderedDims = newOrderedDims ++ complexDims ++ measures
 
     cm.partitioner match {
@@ -1963,11 +1978,19 @@ private[sql] case class DescribeCommandFormatted(
   override def run(sqlContext: SQLContext): Seq[Row] = {
     val relation = CarbonEnv.getInstance(sqlContext).carbonCatalog
       .lookupRelation2(tblIdentifier, None)(sqlContext).asInstanceOf[CarbonRelation]
+    val mapper = new ObjectMapper()
+    val colProps = StringBuilder.newBuilder
     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(
             relation.cubeMeta.carbonTableIdentifier.getTableName,
             field.name)
+        if (null != dimension.getColumnProperties && dimension.getColumnProperties.size() > 0) {
+          val colprop = mapper.writeValueAsString(dimension.getColumnProperties)
+          colProps.append(field.name).append(".")
+          .append(mapper.writeValueAsString(dimension.getColumnProperties))
+          .append(",")
+        }
         if (dimension.hasEncoding(Encoding.DICTIONARY) &&
             !dimension.hasEncoding(Encoding.DIRECT_DICTIONARY)) {
           "DICTIONARY, KEY COLUMN"
@@ -1975,18 +1998,22 @@ private[sql] case class DescribeCommandFormatted(
           "KEY COLUMN"
         }
       } else {
-        ""
+        ("MEASURE")
       }
       (field.name, field.dataType.simpleString, comment)
     }
+    val colPropStr = if (colProps.toString().trim().length() > 0) {
+      // drops additional comma at end
+      colProps.toString().dropRight(1)
+    } else {
+      colProps.toString()
+    }
     results ++= Seq(("", "", ""), ("##Detailed Table Information", "", ""))
     results ++= Seq(("Database Name : ", relation.cubeMeta.carbonTableIdentifier
       .getDatabaseName, "")
     )
     results ++= Seq(("Table Name : ", relation.cubeMeta.carbonTableIdentifier.getTableName, ""))
     results ++= Seq(("CARBON Store Path : ", relation.cubeMeta.storePath, ""))
-    results ++= getColumnGroups(relation.metaData.carbonTable.getDimensionByTableName(
-        relation.cubeMeta.carbonTableIdentifier.getTableName).asScala.toList)
     results ++= Seq(("", "", ""), ("#Aggregate Tables", "", ""))
     val carbonTable = relation.cubeMeta.carbonTable
     val aggTables = carbonTable.getAggregateTablesName
@@ -1994,7 +2021,9 @@ private[sql] case class DescribeCommandFormatted(
       results ++= Seq(("NONE", "", ""))
     } else {
       aggTables.asScala.foreach(aggTable => {
-        results ++= Seq(("", "", ""), ("Agg Table :" + aggTable, "#Columns", "#AggregateType"))
+        results ++= Seq(("", "", ""),
+          ("Agg Table :" + aggTable, "#Columns", "#AggregateType")
+        )
         carbonTable.getDimensionByTableName(aggTable).asScala.foreach(dim => {
           results ++= Seq(("", dim.getColName, ""))
         })
@@ -2004,9 +2033,14 @@ private[sql] case class DescribeCommandFormatted(
       }
       )
     }
-
+    results ++= Seq(("", "", ""), ("##Detailed Column property", "", ""))
+    if (colPropStr.length() > 0) {
+      results ++= Seq((colPropStr, "", ""))
+    } else {
+      results ++= Seq(("NONE", "", ""))
+    }
     results.map { case (name, dataType, comment) =>
-      Row(name, dataType, comment)
+      Row(f"$name%-36s $dataType%-80s $comment%-72s")
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 6fb43a5..412ca90 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
@@ -52,14 +52,18 @@ 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 :+ getCarbonTableScans :+ getDDLStrategies
     total
   }
 
+  def getCarbonTableScans: Strategy = new CarbonTableScans
+
+  def getDDLStrategies: Strategy = new DDLStrategies
+
   /**
    * Carbon strategies for Carbon cube scanning
    */
-  private[sql] object CarbonTableScans extends Strategy {
+  private[sql] class CarbonTableScans extends Strategy {
 
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
       case PhysicalOperation(projectList, predicates,
@@ -321,7 +325,7 @@ class CarbonStrategies(sqlContext: SQLContext) extends QueryPlanner[SparkPlan] {
     }
   }
 
-  object DDLStrategies extends Strategy {
+  class DDLStrategies extends Strategy {
     def apply(plan: LogicalPlan): Seq[SparkPlan] = plan match {
       case ShowCubeCommand(schemaName) =>
         ExecutedCommand(ShowAllTablesInSchema(schemaName, plan.output)) :: Nil

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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..beab9cb 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
@@ -26,10 +26,10 @@ 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)
+      Seq(carbonStrategy.getCarbonTableScans, carbonStrategy.getDDLStrategies)
     } else {
       // TODO: need to remove duplicate code in strategies.
-      Seq(new CarbonRawStrategies(context).CarbonRawTableScans, carbonStrategy.DDLStrategies)
+      Seq(new CarbonRawStrategies(context).CarbonRawTableScans, carbonStrategy.getDDLStrategies)
     }
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/main/scala/org/carbondata/spark/CarbonColumnValidator.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/CarbonColumnValidator.scala b/integration/spark/src/main/scala/org/carbondata/spark/CarbonColumnValidator.scala
new file mode 100644
index 0000000..51c4b67
--- /dev/null
+++ b/integration/spark/src/main/scala/org/carbondata/spark/CarbonColumnValidator.scala
@@ -0,0 +1,36 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one or more
+ * contributor license agreements.  See the NOTICE file distributed with
+ * this work for additional information regarding copyright ownership.
+ * The ASF licenses this file to You under the Apache License, Version 2.0
+ * (the "License"); you may not use this file except in compliance with
+ * the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.carbondata.spark
+
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema
+import org.carbondata.spark.exception.MalformedCarbonCommandException
+
+ /**
+  * Carbon column validator
+  */
+class CarbonColumnValidator extends ColumnValidator {
+  def validateColumns(allColumns: Seq[ColumnSchema]) {
+    allColumns.foreach { columnSchema =>
+      val colWithSameId = allColumns.filter { x =>
+        x.getColumnUniqueId.equals(columnSchema.getColumnUniqueId)
+      }
+      if (colWithSameId.size > 1) {
+        throw new MalformedCarbonCommandException("Two column can not have same columnId")
+      }
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/main/scala/org/carbondata/spark/CarbonSparkFactory.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/CarbonSparkFactory.scala b/integration/spark/src/main/scala/org/carbondata/spark/CarbonSparkFactory.scala
new file mode 100644
index 0000000..80a6ad6
--- /dev/null
+++ b/integration/spark/src/main/scala/org/carbondata/spark/CarbonSparkFactory.scala
@@ -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.spark
+
+import org.carbondata.core.carbon.CarbonTableIdentifier
+import org.carbondata.core.carbon.ColumnIdentifier
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
+import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema
+import org.carbondata.core.carbon.path.CarbonTablePath
+import org.carbondata.spark.exception.MalformedCarbonCommandException
+
+ /**
+  * Column validator
+  */
+trait ColumnValidator {
+  def validateColumns(columns: Seq[ColumnSchema])
+}
+/**
+ * Dictionary related helper service
+ */
+trait DictionaryDetailService {
+  def getDictionaryDetail(dictFolderPath: String, primDimensions: Array[CarbonDimension],
+      table: CarbonTableIdentifier, hdfsLocation: String): DictionaryDetail
+}
+
+/**
+ * Dictionary related detail
+ */
+case class DictionaryDetail(columnIdentifiers: Array[ColumnIdentifier],
+    dictFilePaths: Array[String], dictFileExists: Array[Boolean])
+
+/**
+ * Factory class
+ */
+object CarbonSparkFactory {
+   /**
+    * @return column validator
+    */
+  def getCarbonColumnValidator(): ColumnValidator = {
+    new CarbonColumnValidator
+  }
+
+  /**
+   * @return dictionary helper
+   */
+  def getDictionaryDetailService(): DictionaryDetailService = {
+    new DictionaryDetailHelper
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/main/scala/org/carbondata/spark/DictionaryDetailHelper.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/DictionaryDetailHelper.scala b/integration/spark/src/main/scala/org/carbondata/spark/DictionaryDetailHelper.scala
new file mode 100644
index 0000000..e144f00
--- /dev/null
+++ b/integration/spark/src/main/scala/org/carbondata/spark/DictionaryDetailHelper.scala
@@ -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.spark
+
+import scala.collection.mutable.HashMap
+
+import org.carbondata.common.factory.CarbonCommonFactory
+import org.carbondata.core.carbon.CarbonTableIdentifier
+import org.carbondata.core.carbon.ColumnIdentifier
+import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
+import org.carbondata.core.carbon.path.CarbonStorePath
+import org.carbondata.core.carbon.path.CarbonTablePath
+import org.carbondata.core.datastorage.store.filesystem.CarbonFile
+import org.carbondata.core.datastorage.store.filesystem.CarbonFileFilter
+import org.carbondata.core.datastorage.store.impl.FileFactory
+
+class DictionaryDetailHelper extends DictionaryDetailService {
+  def getDictionaryDetail(dictfolderPath: String, primDimensions: Array[CarbonDimension],
+      table: CarbonTableIdentifier, hdfsLocation: String): DictionaryDetail = {
+    val carbonTablePath = CarbonStorePath.getCarbonTablePath(hdfsLocation, table)
+    val dictFilePaths = new Array[String](primDimensions.length)
+    val dictFileExists = new Array[Boolean](primDimensions.length)
+    val columnIdentifier = new Array[ColumnIdentifier](primDimensions.length)
+
+    val fileType = FileFactory.getFileType(dictfolderPath)
+    // Metadata folder
+    val metadataDirectory = FileFactory.getCarbonFile(dictfolderPath, fileType)
+    // need list all dictionary file paths with exists flag
+    val carbonFiles = metadataDirectory.listFiles(new CarbonFileFilter {
+      @Override def accept(pathname: CarbonFile): Boolean = {
+        CarbonTablePath.isDictionaryFile(pathname)
+      }
+    })
+    // 2 put dictionary file names to fileNamesMap
+    val fileNamesMap = new HashMap[String, Int]
+    for (i <- 0 until carbonFiles.length) {
+      fileNamesMap.put(carbonFiles(i).getName, i)
+    }
+    // 3 lookup fileNamesMap, if file name is in fileNamesMap, file is exists, or not.
+    primDimensions.zipWithIndex.foreach { f =>
+      columnIdentifier(f._2) = f._1.getColumnIdentifier
+      dictFilePaths(f._2) = carbonTablePath.getDictionaryFilePath(f._1.getColumnId)
+      dictFileExists(f._2) =
+        fileNamesMap.get(CarbonTablePath.getDictionaryFileName(f._1.getColumnId)) match {
+          case None => false
+          case Some(_) => true
+        }
+    }
+
+    DictionaryDetail(columnIdentifier, dictFilePaths, dictFileExists)
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
index ca18d24..a78daa8 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonGlobalDictionaryRDD.scala
@@ -29,7 +29,7 @@ import org.apache.spark.rdd.RDD
 import org.apache.spark.sql.Row
 
 import org.carbondata.common.logging.LogServiceFactory
-import org.carbondata.core.carbon.CarbonTableIdentifier
+import org.carbondata.core.carbon.{CarbonTableIdentifier, ColumnIdentifier}
 import org.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension
 import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.spark.load.CarbonLoaderUtil
@@ -140,6 +140,7 @@ case class DictionaryLoadModel(table: CarbonTableIdentifier,
     highCardIdentifyEnable: Boolean,
     highCardThreshold: Int,
     rowCountPercentage: Double,
+    columnIdentifier: Array[ColumnIdentifier],
     isFirstLoad: Boolean) extends Serializable
 
 case class ColumnDistinctValues(values: Array[String], rowCount: Long) extends Serializable
@@ -232,7 +233,7 @@ class CarbonGlobalDictionaryGenerateRDD(
         val t1 = System.currentTimeMillis
         dictionaryForDistinctValueLookUp = if (model.dictFileExists(split.index)) {
           CarbonLoaderUtil.getDictionary(model.table,
-            model.primDimensions(split.index).getColumnId,
+            model.columnIdentifier(split.index),
             model.hdfsLocation,
             model.primDimensions(split.index).getDataType
           )
@@ -275,7 +276,7 @@ class CarbonGlobalDictionaryGenerateRDD(
           val t4 = System.currentTimeMillis
           if (distinctValueCount > 0) {
             dictionaryForSortIndexWriting = CarbonLoaderUtil.getDictionary(model.table,
-              model.primDimensions(split.index).getColumnId,
+              model.columnIdentifier(split.index),
               model.hdfsLocation,
               model.primDimensions(split.index).getDataType)
             GlobalDictionaryUtil.writeGlobalDictionaryColumnSortInfo(model, split.index,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/main/scala/org/carbondata/spark/util/CommonUtil.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/carbondata/spark/util/CommonUtil.scala b/integration/spark/src/main/scala/org/carbondata/spark/util/CommonUtil.scala
index 72a1256..9ede819 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/util/CommonUtil.scala
@@ -16,8 +16,13 @@
  */
 package org.carbondata.spark.util
 
-import org.apache.spark.sql.execution.command.Field
+import java.util
+import java.util.UUID
 
+import org.apache.spark.sql.execution.command.{ColumnProperty, Field}
+
+import org.carbondata.core.carbon.metadata.datatype.DataType
+import org.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.spark.exception.MalformedCarbonCommandException
 
 object CommonUtil {
@@ -54,6 +59,7 @@ object CommonUtil {
 
   }
 
+
   def isTimeStampColumn(colName: String, dims: Seq[Field]): Boolean = {
     dims.foreach { dim =>
       if (dim.column.equalsIgnoreCase(colName)) {
@@ -84,4 +90,75 @@ object CommonUtil {
     }
     false
   }
+
+  def getColumnProperties(column: String,
+      tableProperties: Map[String, String]): Option[util.List[ColumnProperty]] = {
+    val fieldProps = new util.ArrayList[ColumnProperty]()
+    val columnPropertiesStartKey = CarbonCommonConstants.COLUMN_PROPERTIES + "." + column + "."
+    tableProperties.foreach {
+      case (key, value) =>
+        if (key.startsWith(columnPropertiesStartKey)) {
+          fieldProps.add(ColumnProperty(key.substring(columnPropertiesStartKey.length(),
+            key.length()), value))
+        }
+    }
+    if (fieldProps.isEmpty()) {
+      None
+    } else {
+      Some(fieldProps)
+    }
+  }
+
+  def validateTblProperties(tableProperties: Map[String, String], fields: Seq[Field]): Boolean = {
+    val itr = tableProperties.keys
+    var isValid: Boolean = true
+    tableProperties.foreach {
+      case (key, value) =>
+        if (!validateFields(key, fields)) {
+          isValid = false
+          throw new MalformedCarbonCommandException(s"Invalid table properties ${ key }")
+        }
+    }
+    isValid
+  }
+
+  def validateFields(key: String, fields: Seq[Field]): Boolean = {
+    var isValid: Boolean = false
+    fields.foreach { field =>
+      if (field.children.isDefined && field.children.get != null) {
+        field.children.foreach(fields => {
+          fields.foreach(complexfield => {
+            val column = if ("val" == complexfield.column) {
+              field.column
+            } else {
+              field.column + "." + complexfield.column
+            }
+            if (validateColumnProperty(key, column)) {
+              isValid = true
+            }
+          }
+          )
+        }
+        )
+      } else {
+        if (validateColumnProperty(key, field.column)) {
+          isValid = true
+        }
+      }
+
+    }
+    isValid
+  }
+
+  def validateColumnProperty(key: String, column: String): Boolean = {
+    if (!key.startsWith(CarbonCommonConstants.COLUMN_PROPERTIES)) {
+      return true
+    }
+    val columnPropertyKey = CarbonCommonConstants.COLUMN_PROPERTIES + "." + column + "."
+    if (key.startsWith(columnPropertyKey)) {
+      true
+    } else {
+      false
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 6298aaa..77e2eec 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
@@ -33,9 +33,11 @@ import org.apache.spark.sql.{CarbonEnv, CarbonRelation, DataFrame, SQLContext}
 import org.apache.spark.sql.hive.CarbonMetastoreCatalog
 import org.apache.spark.util.FileUtils
 
+import org.carbondata.common.factory.CarbonCommonFactory
 import org.carbondata.core.cache.dictionary.Dictionary
 import org.carbondata.core.carbon.CarbonDataLoadSchema
 import org.carbondata.core.carbon.CarbonTableIdentifier
+import org.carbondata.core.carbon.ColumnIdentifier
 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
@@ -53,6 +55,7 @@ import org.carbondata.spark.load.CarbonLoaderUtil
 import org.carbondata.spark.load.CarbonLoadModel
 import org.carbondata.spark.partition.reader.CSVWriter
 import org.carbondata.spark.rdd.{ArrayParser, CarbonBlockDistinctValuesCombineRDD, CarbonDataRDDFactory, CarbonGlobalDictionaryGenerateRDD, ColumnPartitioner, DataFormat, DictionaryLoadModel, GenericParser, PrimitiveParser, StructParser}
+import org.carbondata.spark.CarbonSparkFactory
 
 /**
  * A object which provide a method to generate global dictionary from CSV files.
@@ -137,9 +140,12 @@ object GlobalDictionaryUtil extends Logging {
   def writeGlobalDictionaryToFile(model: DictionaryLoadModel,
       columnIndex: Int,
       iter: Iterator[String]): Unit = {
-    val writer: CarbonDictionaryWriter = new CarbonDictionaryWriterImpl(
-      model.hdfsLocation, model.table,
-      model.primDimensions(columnIndex).getColumnId)
+    val dictService = CarbonCommonFactory.getDictionaryService
+    val writer: CarbonDictionaryWriter = dictService.getDictionaryWriter(
+      model.table,
+      model.columnIdentifier(columnIndex),
+      model.hdfsLocation
+    )
     try {
       while (iter.hasNext) {
         writer.write(iter.next)
@@ -158,12 +164,13 @@ object GlobalDictionaryUtil extends Logging {
       index: Int,
       dictionary: Dictionary): Unit = {
     val preparator: CarbonDictionarySortInfoPreparator = new CarbonDictionarySortInfoPreparator
+    val dictService = CarbonCommonFactory.getDictionaryService
     val dictionarySortInfo: CarbonDictionarySortInfo =
       preparator.getDictionarySortInfo(dictionary,
         model.primDimensions(index).getDataType)
     val carbonDictionaryWriter: CarbonDictionarySortIndexWriter =
-      new CarbonDictionarySortIndexWriterImpl(model.table,
-        model.primDimensions(index).getColumnId, model.hdfsLocation)
+      dictService.getDictionarySortIndexWriter(model.table, model.columnIdentifier(index),
+          model.hdfsLocation)
     try {
       carbonDictionaryWriter.writeSortIndex(dictionarySortInfo.getSortIndex)
       carbonDictionaryWriter.writeInvertedSortIndex(dictionarySortInfo.getSortIndexInverted)
@@ -179,7 +186,7 @@ object GlobalDictionaryUtil extends Logging {
     val dictMap = new HashMap[String, Dictionary]
     model.primDimensions.zipWithIndex.filter(f => model.dictFileExists(f._2)).foreach { m =>
       val dict = CarbonLoaderUtil.getDictionary(model.table,
-        m._1.getColumnId, model.hdfsLocation,
+        m._1.getColumnIdentifier, model.hdfsLocation,
         m._1.getDataType
       )
       dictMap.put(m._1.getColumnId, dict)
@@ -192,11 +199,13 @@ object GlobalDictionaryUtil extends Logging {
    */
   def readGlobalDictionaryFromFile(model: DictionaryLoadModel): HashMap[String, HashSet[String]] = {
     val dictMap = new HashMap[String, HashSet[String]]
+    val dictService = CarbonCommonFactory.getDictionaryService
     for (i <- model.primDimensions.indices) {
       val set = new HashSet[String]
       if (model.dictFileExists(i)) {
-        val reader: CarbonDictionaryReader = new CarbonDictionaryReaderImpl(
-          model.hdfsLocation, model.table, model.primDimensions(i).getColumnId)
+        val reader: CarbonDictionaryReader = dictService.getDictionaryReader(model.table,
+          model.columnIdentifier(i), model.hdfsLocation
+        )
         val values = reader.read
         if (values != null) {
           for (j <- 0 until values.size) {
@@ -293,52 +302,13 @@ object GlobalDictionaryUtil extends Logging {
         isComplexes += dimensions(i).isComplex
       }
     }
-    val primDimensions = primDimensionsBuffer.map { x => x }.toArray
-    val dictFilePaths = new Array[String](primDimensions.length)
-    val dictFileExists = new Array[Boolean](primDimensions.length)
     val carbonTablePath = CarbonStorePath.getCarbonTablePath(hdfsLocation, table)
-    val fileType = FileFactory.getFileType(dictfolderPath)
-    // Metadata folder
-    val metadataDirectory = FileFactory.getCarbonFile(dictfolderPath, fileType)
-    // need list all dictionary file paths with exists flag
-    metadataDirectory.exists match {
-      case true =>
-        // if Metadata folder is exists, check whether each dictionary file is exists or not.
-        // 1 list all dictionary files in Metadata folder
-        val carbonFiles = metadataDirectory.listFiles(new CarbonFileFilter {
-          @Override def accept(pathname: CarbonFile): Boolean = {
-            CarbonTablePath.isDictionaryFile(pathname)
-          }
-        })
-        // 2 put dictionary file names to fileNamesMap
-        val fileNamesMap = new HashMap[String, Int]
-        for (i <- 0 until carbonFiles.length) {
-          fileNamesMap.put(carbonFiles(i).getName, i)
-        }
-        // 3 lookup fileNamesMap, if file name is in fileNamesMap, file is exists, or not.
-        primDimensions.zipWithIndex.foreach { f =>
-          dictFilePaths(f._2) = carbonTablePath.getDictionaryFilePath(f._1.getColumnId)
-          dictFileExists(f._2) =
-            fileNamesMap.get(CarbonTablePath.getDictionaryFileName(f._1.getColumnId)) match {
-              case None => false
-              case Some(_) => true
-            }
-        }
-      case false =>
-        // if Metadata folder is not exists, all dictionary files are not exists also.
-        try {
-          // create Metadata folder
-          FileFactory.mkdirs(dictfolderPath, fileType)
-        } catch {
-          case ex: IOException =>
-            throw new IOException(s"Failed to created dictionary folder: ${dictfolderPath}")
-        }
-        primDimensions.zipWithIndex.foreach { f =>
-          dictFilePaths(f._2) = carbonTablePath.getDictionaryFilePath(f._1.getColumnId)
-          // all dictionary files are not exists
-          dictFileExists(f._2) = false
-        }
-    }
+    val primDimensions = primDimensionsBuffer.map { x => x }.toArray
+    val dictDetail = CarbonSparkFactory.getDictionaryDetailService().
+    getDictionaryDetail(dictfolderPath, primDimensions, table, hdfsLocation)
+    val dictFilePaths = dictDetail.dictFilePaths
+    val dictFileExists = dictDetail.dictFileExists
+    val columnIdentifier = dictDetail.columnIdentifiers
 
     // load high cardinality identify configure
     val highCardIdentifyEnable = CarbonProperties.getInstance().getProperty(
@@ -367,6 +337,7 @@ object GlobalDictionaryUtil extends Logging {
       highCardIdentifyEnable,
       highCardThreshold,
       rowCountPercentage,
+      columnIdentifier,
       carbonLoadModel.getLoadMetadataDetails.size() == 0)
   }
 
@@ -552,9 +523,11 @@ object GlobalDictionaryUtil extends Logging {
     val values = valuesBuffer.toArray
     java.util.Arrays.sort(values, Ordering[String])
     var distinctValueCount: Int = 0
-    val writer: CarbonDictionaryWriter = new CarbonDictionaryWriterImpl(
-      model.hdfsLocation, model.table,
-      model.primDimensions(columnIndex).getColumnId)
+    val dictService = CarbonCommonFactory.getDictionaryService
+    val writer: CarbonDictionaryWriter = dictService.getDictionaryWriter(
+        model.table,
+        model.columnIdentifier(columnIndex),
+        model.hdfsLocation)
     try {
       if (!model.dictFileExists(columnIndex)) {
         writer.write(CarbonCommonConstants.MEMBER_DEFAULT_VAL)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/test/java/org/carbondata/integration/spark/load/CarbonLoaderUtilTest.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/java/org/carbondata/integration/spark/load/CarbonLoaderUtilTest.java b/integration/spark/src/test/java/org/carbondata/integration/spark/load/CarbonLoaderUtilTest.java
index 88ab188..8399d28 100644
--- a/integration/spark/src/test/java/org/carbondata/integration/spark/load/CarbonLoaderUtilTest.java
+++ b/integration/spark/src/test/java/org/carbondata/integration/spark/load/CarbonLoaderUtilTest.java
@@ -1,4 +1,4 @@
-package org.carbondata.spark.load;
+package org.carbondata.integration.spark.load;
 
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -7,7 +7,7 @@ import java.util.List;
 import java.util.Map;
 
 import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
-
+import org.carbondata.spark.load.CarbonLoaderUtil;
 import org.junit.Test;
 import org.pentaho.di.core.util.Assert;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/test/java/org/carbondata/integration/spark/testsuite/validation/FileFooterValidator.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/java/org/carbondata/integration/spark/testsuite/validation/FileFooterValidator.java b/integration/spark/src/test/java/org/carbondata/integration/spark/testsuite/validation/FileFooterValidator.java
index a254f8d..ccc2820 100644
--- a/integration/spark/src/test/java/org/carbondata/integration/spark/testsuite/validation/FileFooterValidator.java
+++ b/integration/spark/src/test/java/org/carbondata/integration/spark/testsuite/validation/FileFooterValidator.java
@@ -1,4 +1,4 @@
-package org.carbondata.spark.testsuite.validation;
+package org.carbondata.integration.spark.testsuite.validation;
 
 import org.apache.spark.sql.common.util.CarbonHiveContext;
 import org.carbondata.core.carbon.CarbonTableIdentifier;
@@ -15,10 +15,10 @@ import org.carbondata.format.BlockletInfo;
 import org.carbondata.format.DataChunk;
 import org.carbondata.format.Encoding;
 import org.carbondata.format.FileFooter;
-
 import org.junit.AfterClass;
 import org.junit.Before;
 import org.junit.Test;
+
 import static org.junit.Assert.assertTrue;
 
 public class FileFooterValidator {
@@ -44,7 +44,7 @@ public class FileFooterValidator {
                 + "PARTITIONDATA(DELIMITER ',', QUOTECHAR '\"')");
     String storePath =
         CarbonProperties.getInstance().getProperty(CarbonCommonConstants.STORE_LOCATION);
-    CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("default", "validatefooter");
+    CarbonTableIdentifier tableIdentifier = new CarbonTableIdentifier("default", "validatefooter", "1");
     String segmentPath = CarbonStorePath.getCarbonTablePath(storePath, tableIdentifier)
         .getCarbonDataDirectoryPath("0", "0");
     CarbonFile carbonFile =

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala b/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
index 969b01f..f825ce2 100644
--- a/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
+++ b/integration/spark/src/test/scala/org/apache/spark/sql/TestCarbonSqlParser.scala
@@ -2,6 +2,7 @@ package org.apache.spark.sql
 
 import org.apache.spark.sql.common.util.QueryTest
 import org.apache.spark.sql.execution.command.Field
+import org.carbondata.core.constants.CarbonCommonConstants
 
 /**
   * Stub class for calling the CarbonSqlParser
@@ -69,7 +70,7 @@ class TestCarbonSqlParser extends QueryTest {
   // Testing the column group Splitting method.
   test("Test-updateColumnGroupsInField") {
     val colGroupStr = "(col2,col3),(col5,col6),(col7,col8)"
-    val tableProperties = Map("COLUMN_GROUPS" -> colGroupStr)
+    val tableProperties = Map(CarbonCommonConstants.COLUMN_GROUPS -> colGroupStr)
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     val colgrps = stub.updateColumnGroupsInFieldTest(fields, tableProperties)
@@ -80,7 +81,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
   test("Test-ColumnGroupsInvalidField_Shouldnotallow") {
     val colGroupStr = "(col1,col2),(col10,col6),(col7,col8)"
-    val tableProperties = Map("COLUMN_GROUPS" -> colGroupStr)
+    val tableProperties = Map(CarbonCommonConstants.COLUMN_GROUPS -> colGroupStr)
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     try {
@@ -93,7 +94,7 @@ class TestCarbonSqlParser extends QueryTest {
   test("Test-MeasureInColumnGroup_ShouldNotAllow") {
     //col1 is measure
     val colGroupStr = "(col1,col2),(col5,col6),(col7,col8)"
-    val tableProperties = Map("COLUMN_GROUPS" -> colGroupStr)
+    val tableProperties = Map(CarbonCommonConstants.COLUMN_GROUPS -> colGroupStr)
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     try {
@@ -107,7 +108,7 @@ class TestCarbonSqlParser extends QueryTest {
     //col5 is no dictionary
     val colGroupStr = "(col2,col3),(col5,col6),(col7,col8)"
     val noDictStr = "col5"
-    val tableProperties = Map("COLUMN_GROUPS" -> colGroupStr, "DICTIONARY_EXCLUDE" -> noDictStr)
+    val tableProperties = Map(CarbonCommonConstants.COLUMN_GROUPS -> colGroupStr, CarbonCommonConstants.DICTIONARY_EXCLUDE -> noDictStr)
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     try {
@@ -119,7 +120,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
   test("Test-SameColumnInDifferentGroup_ShouldNotAllow") {
     val colGroupStr = "(col2,col3),(col5,col6),(col6,col7,col8)"
-    val tableProperties = Map("COLUMN_GROUPS" -> colGroupStr)
+    val tableProperties = Map(CarbonCommonConstants.COLUMN_GROUPS -> colGroupStr)
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     try {
@@ -132,7 +133,7 @@ class TestCarbonSqlParser extends QueryTest {
   
    test("Test-ColumnAreNotTogetherAsInSchema_ShouldNotAllow") {
     val colGroupStr = "(col2,col3),(col5,col8)"
-    val tableProperties = Map("COLUMN_GROUPS" -> colGroupStr)
+    val tableProperties = Map(CarbonCommonConstants.COLUMN_GROUPS -> colGroupStr)
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     try {
@@ -144,7 +145,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
   test("Test-ColumnInColumnGroupAreShuffledButInSequence") {
     val colGroupStr = "(col2,col3),(col7,col8,col6)"
-    val tableProperties = Map("COLUMN_GROUPS" -> colGroupStr)
+    val tableProperties = Map(CarbonCommonConstants.COLUMN_GROUPS -> colGroupStr)
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     
@@ -164,7 +165,7 @@ class TestCarbonSqlParser extends QueryTest {
 
   // Testing the extracting of Dims and no Dictionary
   test("Test-extractDimColsAndNoDictionaryFields") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col2", "DICTIONARY_INCLUDE" -> "col4")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col2", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col4")
     var fields: Seq[Field] = loadAllFields
 
     val stub = new TestCarbonSqlParserStub()
@@ -184,7 +185,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields1") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col1")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col1")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var (dimCols, noDictionary) = stub
@@ -207,7 +208,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields2") {
-    val tableProperties = Map("DICTIONARY_INCLUDE" -> "col1")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_INCLUDE -> "col1")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var (dimCols, noDictionary) = stub
@@ -229,7 +230,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields3") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col1", "DICTIONARY_INCLUDE" -> "col4")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col1", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col4")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var (dimCols, noDictionary) = stub
@@ -252,7 +253,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields4") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col3", "DICTIONARY_INCLUDE" -> "col2")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col3", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col2")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var (dimCols, noDictionary) = stub
@@ -275,7 +276,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields5") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col4", "DICTIONARY_INCLUDE" -> "col2")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col4", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col2")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var (dimCols, noDictionary) = stub
@@ -298,7 +299,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields6") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col2", "DICTIONARY_INCLUDE" -> "col1")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col2", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col1")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var (dimCols, noDictionary) = stub
@@ -321,8 +322,8 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields7") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col2 ,col1  ",
-      "DICTIONARY_INCLUDE" -> "col3 ,col4 "
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col2 ,col1  ",
+      CarbonCommonConstants.DICTIONARY_INCLUDE -> "col3 ,col4 "
     )
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
@@ -347,7 +348,7 @@ class TestCarbonSqlParser extends QueryTest {
   }
 
   test("Test-DimAndMsrColsWithNoDictionaryFields8") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col2,col4", "DICTIONARY_INCLUDE" -> "col3")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE-> "col2,col4", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col3")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var (dimCols, noDictionary) = stub
@@ -372,7 +373,7 @@ class TestCarbonSqlParser extends QueryTest {
 
   // Testing the extracting of measures
   test("Test-extractMsrColsFromFields") {
-    val tableProperties = Map("DICTIONARY_EXCLUDE" -> "col2", "DICTIONARY_INCLUDE" -> "col4")
+    val tableProperties = Map(CarbonCommonConstants.DICTIONARY_EXCLUDE -> "col2", CarbonCommonConstants.DICTIONARY_INCLUDE -> "col4")
     var fields: Seq[Field] = loadAllFields
     val stub = new TestCarbonSqlParserStub()
     var msrCols = stub.extractMsrColsFromFieldsTest(fields, tableProperties)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
index 9d10937..21a8a46 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithMalformedCarbonCommandException.scala
@@ -87,7 +87,7 @@ class TestLoadDataWithMalformedCarbonCommandException extends QueryTest with Bef
       buildTableWithNoExistDictExclude()
     } catch {
       case e: MalformedCarbonCommandException =>
-        assert(e.getMessage.equals("DICTIONARY_EXCLUDE column: CCC does not exist in table. " +
+        assert(e.getMessage.equals("DICTIONARY_EXCLUDE column: ccc does not exist in table. " +
           "Please check create table statement."))
       case _ => assert(false)
     }
@@ -98,7 +98,7 @@ class TestLoadDataWithMalformedCarbonCommandException extends QueryTest with Bef
       buildTableWithNoExistDictInclude()
     } catch {
       case e: MalformedCarbonCommandException =>
-        assert(e.getMessage.equals("DICTIONARY_INCLUDE column: AAA does not exist in table. " +
+        assert(e.getMessage.equals("DICTIONARY_INCLUDE column: aaa does not exist in table. " +
           "Please check create table statement."))
       case _ => assert(false)
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnPropertyValidationTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnPropertyValidationTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnPropertyValidationTestCase.scala
new file mode 100644
index 0000000..b9affa7
--- /dev/null
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/ColumnPropertyValidationTestCase.scala
@@ -0,0 +1,28 @@
+package org.carbondata.spark.testsuite.detailquery;
+
+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
+
+class ColumnPropertyValidationTestCase extends QueryTest with BeforeAndAfterAll {
+  test("Validate ColumnProperties_ valid key") {
+     try {
+       sql("create table employee(empname String,empid String,city String,country String,gender String,salary Double) stored by 'org.apache.carbondata.format' tblproperties('columnproperties.gender.key'='value')")
+       assert(true)
+       sql("drop table employee")
+     } catch {
+       case e =>assert(false)
+     }
+  }
+  test("Validate Dictionary include _ invalid key") {
+     try {
+       sql("create table employee(empname String,empid String,city String,country String,gender String,salary Double) stored by 'org.apache.carbondata.format' tblproperties('columnproperties.invalid.key'='value')")
+       assert(false)
+       sql("drop table employee")
+     } catch {
+       case e =>assert(true)
+     }
+  }
+  
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 dbe7796..ed50788 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
@@ -25,6 +25,7 @@ import org.apache.spark.sql.common.util.CarbonHiveContext
 import org.apache.spark.sql.common.util.CarbonHiveContext.sql
 import org.apache.spark.sql.common.util.QueryTest
 
+import org.carbondata.core.carbon.ColumnIdentifier
 import org.carbondata.core.cache.dictionary.DictionaryColumnUniqueIdentifier
 import org.carbondata.core.carbon.{CarbonDataLoadSchema, CarbonTableIdentifier}
 import org.carbondata.core.constants.CarbonCommonConstants
@@ -159,10 +160,10 @@ class GlobalDictionaryUtilTestCase extends QueryTest with BeforeAndAfterAll {
     val dimension = table.getDimensionByName(table.getFactTableName, columnName)
     val tableIdentifier = new CarbonTableIdentifier(table.getDatabaseName, table.getFactTableName, "uniqueid")
 
-    val columnIdentifier = new DictionaryColumnUniqueIdentifier(tableIdentifier,
-      dimension.getColumnId, dimension.getDataType
+    val dictColumnIdentifier = new DictionaryColumnUniqueIdentifier(tableIdentifier,
+      dimension.getColumnIdentifier, dimension.getDataType
     )
-    val dict = CarbonLoaderUtil.getDictionary(columnIdentifier,
+    val dict = CarbonLoaderUtil.getDictionary(dictColumnIdentifier,
       CarbonHiveContext.hdfsCarbonBasePath
     )
     assert(dict.getSurrogateKey(value) != CarbonCommonConstants.INVALID_SURROGATE_KEY)

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 d074473..435180b 100644
--- a/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
+++ b/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
@@ -538,6 +538,7 @@ public class GraphGenerator {
     seqMeta.setTaskNo(taskNo);
     seqMeta.setCarbondim(graphConfiguration.getDimensionString());
     seqMeta.setComplexTypeString(graphConfiguration.getComplexTypeString());
+    seqMeta.setColumnPropertiesString(graphConfiguration.getColumnPropertiesString());
     seqMeta.setBatchSize(Integer.parseInt(graphConfiguration.getBatchSize()));
     seqMeta.setNoDictionaryDims(graphConfiguration.getNoDictionaryDims());
     seqMeta.setDimensionColumnsDataType(graphConfiguration.getDimensionColumnsDataType());
@@ -787,6 +788,8 @@ public class GraphGenerator {
         .setDimensions(CarbonSchemaParser.getCubeDimensions(dimensions, carbonDataLoadSchema));
     graphConfiguration
         .setActualDims(CarbonSchemaParser.getCubeDimensions(dimensions, carbonDataLoadSchema));
+    graphConfiguration
+        .setColumnPropertiesString(CarbonSchemaParser.getColumnPropertiesString(dimensions));
     graphConfiguration.setComplexTypeString(CarbonSchemaParser.getComplexTypeString(dimensions));
     prepareNoDictionaryMapping(dimensions, graphConfiguration);
     graphConfiguration

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/processing/src/main/java/org/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java b/processing/src/main/java/org/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
index 46960dc..8ab37a6 100644
--- a/processing/src/main/java/org/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
+++ b/processing/src/main/java/org/carbondata/processing/graphgenerator/configuration/GraphConfigurationInfo.java
@@ -190,6 +190,8 @@ public class GraphConfigurationInfo {
 
   private Boolean[] isNoDictionaryDimMapping;
 
+  private String columnPropertiesString;
+
   /**
    * wrapper object holding the columnschemadetails
    */
@@ -999,6 +1001,14 @@ public class GraphConfigurationInfo {
     this.isNoDictionaryDimMapping = isNoDictionaryDimMapping;
   }
 
+  public void setColumnPropertiesString(String columnPropertiesString) {
+    this.columnPropertiesString = columnPropertiesString;
+  }
+
+  public String getColumnPropertiesString() {
+    return this.columnPropertiesString;
+  }
+
   /**
    * @return columngroups
    */

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/processing/src/main/java/org/carbondata/processing/schema/metadata/ColumnsInfo.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/schema/metadata/ColumnsInfo.java b/processing/src/main/java/org/carbondata/processing/schema/metadata/ColumnsInfo.java
index 5774453..9613a86 100644
--- a/processing/src/main/java/org/carbondata/processing/schema/metadata/ColumnsInfo.java
+++ b/processing/src/main/java/org/carbondata/processing/schema/metadata/ColumnsInfo.java
@@ -154,6 +154,8 @@ public class ColumnsInfo {
    */
   private ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper;
 
+  private Map<String, Map<String, String>> columnProperties;
+
   public Map<String, GenericDataType> getComplexTypesMap() {
     return complexTypesMap;
   }
@@ -510,4 +512,12 @@ public class ColumnsInfo {
   public void setColumnSchemaDetailsWrapper(ColumnSchemaDetailsWrapper columnSchemaDetailsWrapper) {
     this.columnSchemaDetailsWrapper = columnSchemaDetailsWrapper;
   }
+
+  public void setColumnProperties(Map<String, Map<String, String>> columnProperties) {
+    this.columnProperties = columnProperties;
+  }
+
+  public Map<String, String> getColumnProperties(String columnName) {
+    return this.columnProperties.get(columnName);
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
index 4625b4a..77f7297 100644
--- a/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
+++ b/processing/src/main/java/org/carbondata/processing/surrogatekeysgenerator/csvbased/CarbonCSVBasedSeqGenMeta.java
@@ -85,6 +85,8 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
   protected Map<String, GenericDataType> complexTypes =
       new HashMap<String, GenericDataType>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
+  protected Map<String, Map<String, String>> columnProperties =
+      new HashMap<String, Map<String, String>>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
   /**
    * dimLens
    */
@@ -265,6 +267,8 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
   private String complexDelimiterLevel2;
   private String complexTypeString;
 
+  private String columnPropertiesString;
+
   private String[] complexTypeColumns;
   /**
    * Primary Key String
@@ -412,6 +416,10 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
     this.complexTypeString = complexTypeString;
   }
 
+  public void setColumnPropertiesString(String columnPropertiesString) {
+    this.columnPropertiesString = columnPropertiesString;
+  }
+
   public String[] getComplexTypeColumns() {
     return complexTypeColumns;
   }
@@ -617,6 +625,7 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
     carbonhierColumn = "";
     foreignKeyHierarchyString = "";
     complexTypeString = "";
+    columnPropertiesString = "";
     complexDelimiterLevel1 = "";
     complexDelimiterLevel2 = "";
     primaryKeysString = "";
@@ -663,6 +672,8 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
         .append(XMLHandler.addTagValue("foreignKeyHierarchyString", foreignKeyHierarchyString));
     retval.append("    ").append(XMLHandler.addTagValue("complexTypeString", complexTypeString));
     retval.append("    ")
+        .append(XMLHandler.addTagValue("columnPropertiesString", columnPropertiesString));
+    retval.append("    ")
         .append(XMLHandler.addTagValue("complexDelimiterLevel1", complexDelimiterLevel1));
     retval.append("    ")
         .append(XMLHandler.addTagValue("complexDelimiterLevel2", complexDelimiterLevel2));
@@ -719,6 +730,7 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
       carbonhierColumn = XMLHandler.getTagValue(stepnode, "carbonhierColumn");
       foreignKeyHierarchyString = XMLHandler.getTagValue(stepnode, "foreignKeyHierarchyString");
       complexTypeString = XMLHandler.getTagValue(stepnode, "complexTypeString");
+      columnPropertiesString = XMLHandler.getTagValue(stepnode, "columnPropertiesString");
       complexDelimiterLevel1 = XMLHandler.getTagValue(stepnode, "complexDelimiterLevel1");
       complexDelimiterLevel2 = XMLHandler.getTagValue(stepnode, "complexDelimiterLevel2");
       primaryKeysString = XMLHandler.getTagValue(stepnode, "primaryKeysString");
@@ -782,6 +794,9 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
       complexTypeColumns = new String[0];
     }
 
+    if (null != columnPropertiesString) {
+      updateColumnPropertiesMap(columnPropertiesString);
+    }
     hirches = getHierarichies(carbonhier);
 
     hierColumnMap = getHierarchiesColumnMap(carbonhierColumn);
@@ -824,6 +839,23 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
     updateDenormColunList(denormColumNames);
   }
 
+  private void updateColumnPropertiesMap(String columnPropertiesString) {
+    String[] colsProperty = columnPropertiesString.split(CarbonCommonConstants.HASH_SPC_CHARACTER);
+    for (String property : colsProperty) {
+      String[] colKeyVals = property.split(CarbonCommonConstants.COLON_SPC_CHARACTER);
+      String colName = colKeyVals[0];
+      Map<String, String> colPropMap = new HashMap<>();
+      String[] keyVals = colKeyVals[1].split(CarbonCommonConstants.COMA_SPC_CHARACTER);
+      for (int i = 0; i < keyVals.length; i++) {
+        String[] keyVal = keyVals[i].split(CarbonCommonConstants.HYPHEN_SPC_CHARACTER);
+        String key = keyVal[0];
+        String value = keyVal[1];
+        colPropMap.put(key, value);
+      }
+      columnProperties.put(colName, colPropMap);
+    }
+  }
+
   private void updateDenormColunList(String denormColumNames) {
     //
     if (null == denormColumNames || "".equals(denormColumNames)) {
@@ -1634,6 +1666,10 @@ public class CarbonCSVBasedSeqGenMeta extends BaseStepMeta implements StepMetaIn
     return taskNo;
   }
 
+  public Map<String, Map<String, String>> getColumnPropertiesMap() {
+    return columnProperties;
+  }
+
   /**
    * returns wrapper object having the columnSchemaDetails
    *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/38d84e0e/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 d5c8e61..f4e333e 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
@@ -367,6 +367,7 @@ public class CarbonCSVBasedSeqGenStep extends BaseStep {
           columnsInfo.setComplexTypesMap(meta.getComplexTypes());
           columnsInfo.setDimensionColumnIds(meta.getDimensionColumnIds());
           columnsInfo.setColumnSchemaDetailsWrapper(meta.getColumnSchemaDetailsWrapper());
+          columnsInfo.setColumnProperties(meta.getColumnPropertiesMap());
           updateBagLogFileName();
           String key = meta.getSchemaName() + '/' + meta.getCubeName() + '_' + meta.getTableName();
           badRecordslogger = new BadRecordslogger(key, csvFilepath, getBadLogStoreLocation(


[04/50] [abbrv] incubator-carbondata git commit: [Issue - CARBONDATA-9] Fixed No lease issue on the bad record log file. (#742)

Posted by ch...@apache.org.
[Issue - CARBONDATA-9] Fixed No lease issue on the bad record log file. (#742)



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

Branch: refs/heads/master
Commit: f17319d9ee9dec2f4fee34a41aa3f5f08800edff
Parents: 6eb5728
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Sat Jun 25 12:24:32 2016 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sat Jun 25 12:24:32 2016 +0530

----------------------------------------------------------------------
 .../csvbased/CarbonCSVBasedSeqGenStep.java                       | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/f17319d9/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 bc81baa..39335d1 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
@@ -369,8 +369,8 @@ public class CarbonCSVBasedSeqGenStep extends BaseStep {
           columnsInfo.setColumnSchemaDetailsWrapper(meta.getColumnSchemaDetailsWrapper());
           updateBagLogFileName();
           String key = meta.getSchemaName() + '/' + meta.getCubeName() + '_' + meta.getTableName();
-          badRecordslogger = new BadRecordslogger(key, csvFilepath,
-              getBadLogStoreLocation(meta.getSchemaName() + '/' + meta.getCubeName()));
+          badRecordslogger = new BadRecordslogger(key, csvFilepath, getBadLogStoreLocation(
+              meta.getSchemaName() + '/' + meta.getCubeName() + "/" + meta.getTaskNo()));
 
           columnsInfo.setTimeOrdinalIndices(meta.timeOrdinalIndices);
           surrogateKeyGen = new FileStoreSurrogateKeyGenForCSV(columnsInfo, meta.getPartitionID(),


[37/50] [abbrv] incubator-carbondata git commit: [BUG] Min Max select query on decimal columns are failing with ClassCastException in Spark beeline (#774)

Posted by ch...@apache.org.
[BUG] Min Max select query on decimal columns are failing with ClassCastException in Spark beeline (#774)



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

Branch: refs/heads/master
Commit: 5fdadba79743bdf8e03ce6f86267774f6abce55f
Parents: 697ba7b
Author: nareshpr <pr...@gmail.com>
Authored: Thu Jun 30 00:39:50 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Thu Jun 30 00:39:50 2016 +0530

----------------------------------------------------------------------
 .../carbondata/spark/agg/CarbonAggregates.scala | 61 +++++++++++++++++---
 .../testsuite/bigdecimal/TestBigDecimal.scala   | 10 ++++
 2 files changed, 63 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5fdadba7/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
index d12a081..61cf350 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
@@ -513,10 +513,25 @@ case class MaxFunctionCarbon(expr: Expression, base: AggregateExpression1, final
     val agg = resolution match {
       case s: MeasureAggregator => s
       case s =>
-        val dc = new MaxAggregator
+        var dc: MeasureAggregator = null
         if (s != null) {
-          dc.agg(s.toString.toDouble)
-          dc.setNewValue(s.toString.toDouble)
+          s match {
+            case bd: java.math.BigDecimal =>
+              dc = new MaxBigDecimalAggregator
+              dc.agg(new java.math.BigDecimal(s.toString))
+              dc.setNewValue(new java.math.BigDecimal(s.toString))
+            case l: Long =>
+              dc = new MaxLongAggregator
+              dc.agg(s.toString.toLong)
+              dc.setNewValue(s.toString.toLong)
+            case _ =>
+              dc = new MaxAggregator
+              dc.agg(s.toString.toDouble)
+              dc.setNewValue(s.toString.toDouble)
+          }
+        }
+        else {
+          dc = new MaxAggregator
         }
         dc
     }
@@ -532,7 +547,14 @@ case class MaxFunctionCarbon(expr: Expression, base: AggregateExpression1, final
       if (max.isFirstTime) {
         null
       } else {
-        Cast(Literal(max.getValueObject), base.dataType).eval(null)
+        max match {
+          case s: MaxBigDecimalAggregator =>
+            Cast(Literal(max.getBigDecimalValue), base.dataType).eval(null)
+          case s: MaxLongAggregator =>
+            Cast(Literal(max.getLongValue), base.dataType).eval(null)
+          case _ =>
+            Cast(Literal(max.getValueObject), base.dataType).eval(null)
+        }
       }
     } else {
       max
@@ -557,13 +579,29 @@ case class MinFunctionCarbon(expr: Expression, base: AggregateExpression1, final
     val agg = resolution match {
       case s: MeasureAggregator => s
       case s =>
-        val dc: MeasureAggregator = new MinAggregator
+        var dc: MeasureAggregator = null
         if (s != null) {
-          dc.agg(s.toString.toDouble)
-          dc.setNewValue(s.toString.toDouble)
+          s match {
+            case bd: java.math.BigDecimal =>
+              dc = new MinBigDecimalAggregator
+              dc.agg(new java.math.BigDecimal(s.toString))
+              dc.setNewValue(new java.math.BigDecimal(s.toString))
+            case l: Long =>
+              dc = new MinLongAggregator
+              dc.agg(s.toString.toLong)
+              dc.setNewValue(s.toString.toLong)
+            case _ =>
+              dc = new MinAggregator
+              dc.agg(s.toString.toDouble)
+              dc.setNewValue(s.toString.toDouble)
+          }
+        }
+        else {
+          dc = new MinAggregator
         }
         dc
     }
+
     if (min == null) {
       min = agg
     } else {
@@ -576,7 +614,14 @@ case class MinFunctionCarbon(expr: Expression, base: AggregateExpression1, final
       if (min.isFirstTime) {
         null
       } else {
-        Cast(Literal(min.getValueObject), base.dataType).eval(null)
+        min match {
+          case s: MinBigDecimalAggregator =>
+            Cast(Literal(min.getBigDecimalValue), base.dataType).eval(null)
+          case s: MinLongAggregator =>
+            Cast(Literal(min.getLongValue), base.dataType).eval(null)
+          case _ =>
+            Cast(Literal(min.getValueObject), base.dataType).eval(null)
+        }
       }
     } else {
       min

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5fdadba7/integration/spark/src/test/scala/org/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
index 9818515..c837f7b 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/bigdecimal/TestBigDecimal.scala
@@ -61,7 +61,17 @@ class TestBigDecimal extends QueryTest with BeforeAndAfterAll {
     checkAnswer(sql("select min(salary) from carbonTable"),
       sql("select min(salary) from hiveTable"))
   }
+  
+  test("test min datatype on big decimal column") {
+    val output = sql("select min(salary) from carbonTable").collectAsList().get(0).get(0)
+    assert(output.isInstanceOf[java.math.BigDecimal])
+  }
 
+  test("test max datatype on big decimal column") {
+    val output = sql("select max(salary) from carbonTable").collectAsList().get(0).get(0)
+    assert(output.isInstanceOf[java.math.BigDecimal])
+  }
+  
   test("test count function on big decimal column") {
     checkAnswer(sql("select count(salary) from carbonTable"),
       sql("select count(salary) from hiveTable"))


[43/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/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 072d527,0000000..bbef2d5
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,242 -1,0 +1,237 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.
 +        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)) {
++      if (!columnList.get(0).getDimension().hasEncoding(Encoding.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
++   * @throws QueryExecutionException
 +   */
 +  @Override public void getEndKey(SegmentProperties segmentProperties,
 +      AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
-       SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
++      SortedMap<Integer, byte[]> setOfEndKeyByteArray) throws QueryExecutionException {
 +    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();
-       }
++      FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
++          absoluteTableIdentifier, endKeys, segmentProperties);
++      FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
++          setOfEndKeyByteArray);
 +    }
 +  }
 +
 +  /**
 +   * 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/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
index 620b1ba,0000000..4981126
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
@@@ -1,98 -1,0 +1,101 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.executor.exception.QueryExecutionException;
 +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
++   * @throws QueryExecutionException
 +   */
 +  void getEndKey(SegmentProperties segmentProperties, AbsoluteTableIdentifier tableIdentifier,
-       long[] endKeys, SortedMap<Integer, byte[]> setOfEndKeyByteArray);
++      long[] endKeys, SortedMap<Integer, byte[]> setOfEndKeyByteArray)
++      throws QueryExecutionException;
 +
 +  /**
 +   * 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/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
index 48c43be,0000000..4fd9223
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
@@@ -1,110 -1,0 +1,111 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.executor.exception.QueryExecutionException;
 +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) {
++      SortedMap<Integer, byte[]> setOfEndKeyByteArray) throws QueryExecutionException {
 +  }
 +
 +  @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/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
index def2725,0000000..a04e5b4
mode 100644,000000..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
@@@ -1,203 -1,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;
 +
 +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.expression.exception.FilterUnsupportedException;
 +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
++   * @throws FilterUnsupportedException
 +   */
-   @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier) {
++  @Override public void resolve(AbsoluteTableIdentifier absoluteTableIdentifier)
++      throws FilterUnsupportedException {
 +
 +    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.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.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/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
index d09bab5,0000000..cf2a998
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@@ -1,233 -1,0 +1,284 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + */
 +
 +package org.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.common.logging.LogService;
++import org.carbondata.common.logging.LogServiceFactory;
 +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.CarbonMeasure;
 +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.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.exception.FilterIllegalMemberException;
++import org.carbondata.scan.expression.exception.FilterUnsupportedException;
 +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 static final LogService LOGGER =
++      LogServiceFactory.getLogService(RowLevelRangeFilterResolverImpl.class.getName());
 +  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
++   * This method will return the filter values which is present in the range level
 +   * conditional expressions.
 +   *
 +   * @return
 +   */
-   public byte[][] getFilterRangeValues() {
-     List<byte[]> filterValuesList = new ArrayList<byte[]>();
-     if (null != dimColEvaluatorInfoList.get(0).getFilterValues()) {
-       filterValuesList =
++  public byte[][] getFilterRangeValues(SegmentProperties segmentProperties) {
++
++    if (null != dimColEvaluatorInfoList.get(0).getFilterValues() && !dimColEvaluatorInfoList.get(0)
++        .getDimension().hasEncoding(Encoding.DICTIONARY)) {
++      List<byte[]> noDictFilterValuesList =
 +          dimColEvaluatorInfoList.get(0).getFilterValues().getNoDictionaryFilterValuesList();
-       return filterValuesList.toArray((new byte[filterValuesList.size()][]));
++      return noDictFilterValuesList.toArray((new byte[noDictFilterValuesList.size()][]));
++    } else if (null != dimColEvaluatorInfoList.get(0).getFilterValues() && dimColEvaluatorInfoList
++        .get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
++      return FilterUtil.getKeyArray(this.dimColEvaluatorInfoList.get(0).getFilterValues(),
++          this.dimColEvaluatorInfoList.get(0).getDimension(),
++          segmentProperties.getDimensionKeyGenerator());
 +    }
-     return filterValuesList.toArray((new byte[filterValuesList.size()][]));
++    return null;
 +
 +  }
 +
 +  /**
 +   * 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.getStartKey(dimColEvaluatorInfoList.get(0), segmentProperties, startKey);
 +      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);
++    boolean invalidRowsPresent = false;
 +    for (ExpressionResult result : listOfExpressionResults) {
-       if (result.getString() == null) {
-         filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL.getBytes());
-         continue;
++      try {
++        if (result.getString() == null) {
++          filterValuesList.add(CarbonCommonConstants.MEMBER_DEFAULT_VAL.getBytes());
++          continue;
++        }
++        filterValuesList.add(result.getString().getBytes());
++      } catch (FilterIllegalMemberException e) {
++        // Any invalid member while evaluation shall be ignored, system will log the
++        // error only once since all rows the evaluation happens so inorder to avoid
++        // too much log inforation only once the log will be printed.
++        FilterUtil.logError(e, invalidRowsPresent);
 +      }
-       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());
++          if (columnExpression.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
++            filterInfo.setFilterList(getDirectSurrogateValues(columnExpression));
++          } else {
++            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);
 +        }
 +      }
 +    }
 +  }
 +
++  private List<Integer> getDirectSurrogateValues(ColumnExpression columnExpression) {
++    List<ExpressionResult> listOfExpressionResults = new ArrayList<ExpressionResult>(20);
++    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
++        .getDirectDictionaryGenerator(columnExpression.getDimension().getDataType());
++
++    if (this.getFilterExpression() instanceof BinaryConditionalExpression) {
++      listOfExpressionResults =
++          ((BinaryConditionalExpression) this.getFilterExpression()).getLiterals();
++    }
++    List<Integer> filterValuesList = new ArrayList<Integer>(20);
++    try {
++      // if any filter member provided by user is invalid throw error else
++      // system can display inconsistent result.
++      for (ExpressionResult result : listOfExpressionResults) {
++        filterValuesList.add(directDictionaryGenerator
++            .generateDirectSurrogateKey(result.getString(),
++                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
++      }
++    } catch (FilterIllegalMemberException e) {
++      new FilterUnsupportedException(e);
++    }
++    return filterValuesList;
++  }
++
 +  /**
 +   * 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/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
index 273bdf1,0000000..5131cca
mode 100644,000000..100644
--- 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
@@@ -1,80 -1,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.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.constants.CarbonCommonConstants;
 +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.FilterIllegalMemberException;
 +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
++   * @throws FilterUnsupportedException,if exception occurs while evaluating
++   * filter models.
 +   */
 +  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
 +      FilterResolverMetadata metadata) throws FilterUnsupportedException {
 +    DimColumnFilterInfo resolvedFilterObject = null;
 +
-     List<String> evaluateResultListFinal =
-         metadata.getExpression().evaluate(null).getListAsString();
++    List<String> evaluateResultListFinal;
++    try {
++      evaluateResultListFinal = metadata.getExpression().evaluate(null).getListAsString();
++    } catch (FilterIllegalMemberException e) {
++      throw new FilterUnsupportedException(e);
++    }
 +    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));
++      surrogates.add(directDictionaryGenerator.generateDirectSurrogateKey(filterMember,
++          CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
 +    }
 +    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/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
index 7ec2751,0000000..f531474
mode 100644,000000..100644
--- 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
@@@ -1,58 -1,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.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.FilterIllegalMemberException;
 +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 FilterUnsupportedException,if exception occurs while evaluating
++   * filter models.
 +   * @throws QueryExecutionException
 +   */
 +  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
 +      FilterResolverMetadata metadata) throws FilterUnsupportedException {
 +    DimColumnFilterInfo resolvedFilterObject = null;
-     List<String> evaluateResultListFinal =
-         metadata.getExpression().evaluate(null).getListAsString();
++    List<String> evaluateResultListFinal;
++    try {
++      evaluateResultListFinal = metadata.getExpression().evaluate(null).getListAsString();
++    } catch (FilterIllegalMemberException e) {
++      throw new FilterUnsupportedException(e);
++    }
 +    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/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
index ee3d157,0000000..6958d61
mode 100644,000000..100644
--- 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
@@@ -1,56 -1,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.resolverinfo.visitor;
 +
 +import java.util.List;
 +
 +import org.carbondata.common.logging.LogService;
 +import org.carbondata.common.logging.LogServiceFactory;
++import org.carbondata.scan.expression.exception.FilterIllegalMemberException;
 +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
++   * @throws FilterUnsupportedException,if exception occurs while evaluating
++   * filter models.
 +   */
 +  public void populateFilterResolvedInfo(DimColumnResolvedFilterInfo visitableObj,
 +      FilterResolverMetadata metadata) throws FilterUnsupportedException {
 +    DimColumnFilterInfo resolvedFilterObject = null;
-     List<String> evaluateResultListFinal =
-         metadata.getExpression().evaluate(null).getListAsString();
++    List<String> evaluateResultListFinal;
++    try {
++      evaluateResultListFinal = metadata.getExpression().evaluate(null).getListAsString();
++    } catch (FilterIllegalMemberException e) {
++      throw new FilterUnsupportedException(e);
++    }
 +    resolvedFilterObject = FilterUtil
 +        .getNoDictionaryValKeyMemberForFilter(metadata.getTableIdentifier(),
 +            metadata.getColumnExpression(), evaluateResultListFinal, metadata.isIncludeFilter());
 +    visitableObj.setFilterValues(resolvedFilterObject);
 +  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/result/iterator/RawResultIterator.java
index 62849ee,0000000..0f9c339
mode 100644,000000..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
@@@ -1,167 -1,0 +1,169 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.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.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 = null;
 +        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 = null;
 +      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/7f722186/core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
index 5654b1d,0000000..86828f0
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
+++ b/core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
@@@ -1,127 -1,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.util;
 +
 +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 || data.isEmpty()) {
++    if (null == data || data.isEmpty() || CarbonCommonConstants.MEMBER_DEFAULT_VAL.equals(data)) {
 +      return null;
 +    }
 +    try {
 +      switch (actualDataType) {
 +        case INT:
 +          return Integer.parseInt(data);
 +        case SHORT:
 +          return Short.parseShort(data);
 +        case DOUBLE:
 +          return Double.parseDouble(data);
 +        case LONG:
 +          return Long.parseLong(data);
 +        case TIMESTAMP:
 +          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:
 +          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;
 +    }
 +
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/format/src/main/thrift/schema.thrift
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala
----------------------------------------------------------------------
diff --cc integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala
index bd7b596,2471e41..0c13a3a
--- 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
@@@ -40,83 -40,53 +40,83 @@@ class AllDataTypesTestCase1 extends Que
      CarbonProperties.getInstance().addProperty("carbon.direct.surrogate", "false")
      try {
        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 numeric," +
-           "contractNumber numeric) OPTIONS (PARTITIONER [CLASS = 'org.carbondata" +
-           ".spark.partition.api.impl.SampleDataPartitionerImpl' ,COLUMNS= (imei) , " +
-           "PARTITION_COUNT=2] )"
+         "create table Carbon_automation_test (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 " +
+         "string,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) stored by 'org.apache" +
+         ".carbondata.format' TBLPROPERTIES('DICTIONARY_EXCLUDE'='Latest_DAY,deviceInformationId," +
+         "Latest_MONTH')")
+       sql("LOAD DATA LOCAL INPATH '" + currentDirectory +
+           "/src/test/resources/100_olap.csv' INTO table Carbon_automation_test OPTIONS" +
+           "('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_EMUIVersion,Latest_operaSysVersion," +
+           "Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer," +
+           "Latest_webTypeDataVerNumber,Latest_operatorsVersion," +
+           "Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')")
  
 +      )
-       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_EMUIVersion,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,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 ")
+       case e: Exception => print("ERROR : " + e.getMessage)
      }
    }
  
    override def afterAll {
-     sql("drop cube Carbon_automation_test")
+     sql("drop table Carbon_automation_test")
 +    sql("drop table Carbon_automation_hive")
  
    }
  

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/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 da328a1,1707e15..26453cb
--- 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
@@@ -71,60 -67,62 +67,89 @@@ class AllDataTypesTestCase2 extends Que
          .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_test2 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_EMUIVersion,Latest_operaSysVersion,Latest_BacVerNumber,Latest_BacFlashVer," +
-         "Latest_webUIVersion,Latest_webUITypeCarrVer,Latest_webTypeDataVerNumber," +
-         "Latest_operatorsVersion,Latest_phonePADPartitionedVersions,Latest_operatorId," +
-         "gamePointId,gamePointDescription')")
+       sql("LOAD DATA LOCAL INPATH '" + currentDirectory +
+           "/src/test/resources/100_olap.csv' INTO table Carbon_automation_test2 OPTIONS" +
+           "('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_EMUIVersion,Latest_operaSysVersion," +
+           "Latest_BacVerNumber,Latest_BacFlashVer,Latest_webUIVersion,Latest_webUITypeCarrVer," +
+           "Latest_webTypeDataVerNumber,Latest_operatorsVersion," +
+           "Latest_phonePADPartitionedVersions,Latest_operatorId,gamePointId,gamePointDescription')")
+ 
+       sql(
+         "create table 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 " +
+         "string,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 ")
 +
 +      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 ")
+       case e: Exception => print("ERROR : " + e.getMessage)
      }
    }
  
    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_test2")
+       sql("drop table Carbon_automation_hive")
 +      sql("drop table Carbon_automation_hive2")
      } catch {
-       case e: Exception => print("ERROR: DROP Carbon_automation_test2 ")
+       case e: Exception => print("ERROR : " + e.getMessage)
      }
    }
  

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase6.scala
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------



[25/50] [abbrv] incubator-carbondata git commit: Fixed count star with filter issue (#762)

Posted by ch...@apache.org.
Fixed count star with filter issue (#762)



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

Branch: refs/heads/master
Commit: c6cafaa817ddcc848de9421efcb63d9a912b963b
Parents: e88529f
Author: Kumar Vishal <ku...@gmail.com>
Authored: Mon Jun 27 22:15:49 2016 +0800
Committer: Vimal-Das <vi...@gmail.com>
Committed: Mon Jun 27 07:15:49 2016 -0700

----------------------------------------------------------------------
 .../query/carbon/executor/util/QueryUtil.java   | 42 ++++++++++----------
 .../HighCardinalityDataTypesTestCase.scala      |  7 ++++
 2 files changed, 29 insertions(+), 20 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/c6cafaa8/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 84cb975..a7d8f05 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
@@ -257,14 +257,14 @@ public class QueryUtil {
    * 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 dimAggInfo                 dimension present in the dimension aggregation
-   *                                   dictionary will be used to convert to actual data
-   *                                   for aggregation
-   * @param customAggExpression        dimension which is present in the expression for aggregation
-   *                                   we need dictionary data
-   * @param absoluteTableIdentifier    absolute table identifier
+   * @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 customAggExpression     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
    */
@@ -715,28 +715,24 @@ public class QueryUtil {
         aggregatorStartIndex += numberOfAggregatorForColumnGroup;
         continue;
       } else {
-        if(CarbonUtil.hasEncoding(dim.getEncoder(), Encoding.DIRECT_DICTIONARY)){
+        if (CarbonUtil.hasEncoding(dim.getEncoder(), Encoding.DIRECT_DICTIONARY)) {
           dimensionDataAggregators.add(
-              new DirectDictionaryDimensionAggregator(entry.getValue().get(0),
-                  aggregatorStartIndex,
+              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)) {
+        else if (CarbonUtil.hasEncoding(dim.getEncoder(), Encoding.DICTIONARY)) {
           dimensionDataAggregators.add(
               new FixedLengthDimensionAggregator(entry.getValue().get(0), null,
-                  columnUniqueIdToDictionaryMap.get(dim.getColumnId()),
-                  aggregatorStartIndex,
+                  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, dimensionToBlockIndexMapping.get(dim.getOrdinal())));
         }
         aggregatorStartIndex += entry.getValue().get(0).getAggList().size();
       }
@@ -835,10 +831,17 @@ public class QueryUtil {
     // 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
+      // first need to check any measure is present or not and as if measure
+      // if measure is present and if first measure is not a default
+      // measure than add measure otherwise
       // than add first dimension as a measure
+      //as currently if measure is not present then
+      //we are adding default measure so first condition will
+      //never come false but if in future we can remove so not removing first if check
       if (queryMeasure.getColumnName().equals("count(*)")) {
-        if (carbonTable.getMeasureByTableName(tableName).size() > 0) {
+        if (carbonTable.getMeasureByTableName(tableName).size() > 0 && !carbonTable
+            .getMeasureByTableName(tableName).get(0).getColName()
+            .equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)) {
           queryMeasure.setMeasure(carbonTable.getMeasureByTableName(tableName).get(0));
         } else {
           CarbonMeasure dummyMeasure = new CarbonMeasure(
@@ -854,7 +857,6 @@ public class QueryUtil {
     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/c6cafaa8/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
index 1d4c641..a357b31 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
@@ -85,6 +85,13 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
 
   }
 
+  test("Count (*) with filter") {
+    checkAnswer(
+      sql("select count(*) from NO_DICTIONARY_CARBON_6 where empno=11"),
+      Seq(Row(1))
+    )
+  }
+
   test("Detail Query with NO_DICTIONARY_COLUMN Compare With HIVE RESULT") {
 
 


[06/50] [abbrv] incubator-carbondata git commit: [Issue - CARBONDATA-10] Avoid to much logging of timestamp parsing exception in TimeStampDirectDictionaryGenerator (#745)

Posted by ch...@apache.org.
[Issue - CARBONDATA-10] Avoid to much logging of timestamp parsing exception in TimeStampDirectDictionaryGenerator (#745)



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

Branch: refs/heads/master
Commit: d5636db0de30d6461381b58d0c2e594470d9e938
Parents: a174f69
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Sat Jun 25 12:59:38 2016 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sat Jun 25 12:59:38 2016 +0530

----------------------------------------------------------------------
 .../timestamp/TimeStampDirectDictionaryGenerator.java         | 7 ++++---
 1 file changed, 4 insertions(+), 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/d5636db0/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 700a6ee..8493991 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
@@ -33,8 +33,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.
@@ -95,6 +93,9 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
         Date dateToStr = timeParser.parse(cutOffTimeStampString);
         cutOffTimeStampLocal = dateToStr.getTime();
       } catch (ParseException e) {
+        LOGGER.warn("Cannot convert" + cutOffTimeStampString
+            + " to Time/Long type value. Value considered for cutOffTimeStamp is -1." + e
+            .getMessage());
         cutOffTimeStampLocal = -1;
       }
     }
@@ -122,7 +123,7 @@ public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGener
     try {
       dateToStr = timeParser.parse(memberStr);
     } catch (ParseException e) {
-      LOGGER.error("Cannot convert" + TIMESTAMP.toString()
+      LOGGER.debug("Cannot convert " + memberStr
           + " to Time/Long type value. Value considered as null." + e.getMessage());
       dateToStr = null;
     }


[12/50] [abbrv] incubator-carbondata git commit: [Bug] fix bug when high.cardinality.threshold == HIGH_CARDINALITY_THRESHOLD_MIN (#750)

Posted by ch...@apache.org.
[Bug] fix bug when high.cardinality.threshold == HIGH_CARDINALITY_THRESHOLD_MIN (#750)

[Bug] fix bug when high.cardinality.threshold == HIGH_CARDINALITY_THRESHOLD_MIN (#750)

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

Branch: refs/heads/master
Commit: 322a77bf41823d4696c4525dc94f068e23efecd0
Parents: 6049017
Author: Gin-zhj <zh...@huawei.com>
Authored: Sat Jun 25 18:35:15 2016 +0800
Committer: david <Qi...@users.noreply.github.com>
Committed: Sat Jun 25 18:35:15 2016 +0800

----------------------------------------------------------------------
 .../main/java/org/carbondata/core/util/CarbonProperties.java | 8 ++++----
 1 file changed, 4 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/322a77bf/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
index e6a1ce9..c6a3a1b 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonProperties.java
@@ -247,12 +247,12 @@ public final class CarbonProperties {
         CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_DEFAULT);
     try {
       int highcardThreshold = Integer.parseInt(highcardThresholdStr);
-      if(highcardThreshold <= CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_MIN){
+      if(highcardThreshold < CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_MIN){
         LOGGER.info("The high cardinality threshold value \"" + highcardThresholdStr
-            + "\" is invalid. Using the default value \""
-            + CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_DEFAULT);
+            + "\" is invalid. Using the min value \""
+            + CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_MIN);
         carbonProperties.setProperty(CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD,
-            CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_DEFAULT);
+            CarbonCommonConstants.HIGH_CARDINALITY_THRESHOLD_MIN + "");
       }
     } catch (NumberFormatException e) {
       LOGGER.info("The high cardinality threshold value \"" + highcardThresholdStr


[26/50] [abbrv] incubator-carbondata git commit: [Bug] Executor Btree loading performance improvement (#763)

Posted by ch...@apache.org.
[Bug] Executor Btree loading performance improvement (#763)

Make block level btree loading in executor multi threaded.

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

Branch: refs/heads/master
Commit: e689092960045634ca3531918022d5030274eb1c
Parents: c6cafaa
Author: Kumar Vishal <ku...@gmail.com>
Authored: Tue Jun 28 00:09:23 2016 +0800
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Mon Jun 27 21:39:23 2016 +0530

----------------------------------------------------------------------
 .../core/carbon/datastore/BlockIndexStore.java  | 184 +++++++++++++------
 1 file changed, 131 insertions(+), 53 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e6890929/core/src/main/java/org/carbondata/core/carbon/datastore/BlockIndexStore.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/BlockIndexStore.java b/core/src/main/java/org/carbondata/core/carbon/datastore/BlockIndexStore.java
index c24e6a9..7ef7a24 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/BlockIndexStore.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/BlockIndexStore.java
@@ -23,7 +23,13 @@ import java.util.Arrays;
 import java.util.Collections;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.Callable;
 import java.util.concurrent.ConcurrentHashMap;
+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;
@@ -34,6 +40,7 @@ import org.carbondata.core.carbon.datastore.block.TableBlockInfo;
 import org.carbondata.core.carbon.datastore.exception.IndexBuilderException;
 import org.carbondata.core.carbon.metadata.blocklet.DataFileFooter;
 import org.carbondata.core.constants.CarbonCommonConstants;
+import org.carbondata.core.util.CarbonProperties;
 import org.carbondata.core.util.CarbonUtil;
 import org.carbondata.core.util.CarbonUtilException;
 
@@ -99,16 +106,24 @@ public class BlockIndexStore {
    */
   public List<AbstractIndex> loadAndGetBlocks(List<TableBlockInfo> tableBlocksInfos,
       AbsoluteTableIdentifier absoluteTableIdentifier) throws IndexBuilderException {
-    List<AbstractIndex> loadedBlocksList =
-        new ArrayList<AbstractIndex>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+    AbstractIndex[] loadedBlock = new AbstractIndex[tableBlocksInfos.size()];
     addTableLockObject(absoluteTableIdentifier);
-    // sort the block infos
+    // sort the block info
     // so block will be loaded in sorted order this will be required for
     // query execution
     Collections.sort(tableBlocksInfos);
     // get the instance
     Object lockObject = tableLockMap.get(absoluteTableIdentifier);
     Map<TableBlockInfo, AbstractIndex> tableBlockMapTemp = null;
+    int numberOfCores = 1;
+    try {
+      numberOfCores = Integer.parseInt(CarbonProperties.getInstance()
+          .getProperty(CarbonCommonConstants.NUM_CORES,
+              CarbonCommonConstants.NUM_CORES_DEFAULT_VAL));
+    } catch (NumberFormatException e) {
+      numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL);
+    }
+    ExecutorService executor = Executors.newFixedThreadPool(numberOfCores);
     // Acquire the lock to ensure only one query is loading the table blocks
     // if same block is assigned to both the queries
     synchronized (lockObject) {
@@ -120,63 +135,102 @@ public class BlockIndexStore {
       }
     }
     AbstractIndex tableBlock = null;
-    DataFileFooter footer = null;
-    try {
-      for (TableBlockInfo blockInfo : tableBlocksInfos) {
-        // if table block is already loaded then do not load
-        // that block
-        tableBlock = tableBlockMapTemp.get(blockInfo);
-        // if block is not loaded
-        if (null == tableBlock) {
-          // check any lock object is present in
-          // block info lock map
-          Object blockInfoLockObject = blockInfoLock.get(blockInfo);
-          // if lock object is not present then acquire
-          // the lock in block info lock and add a lock object in the map for
-          // particular block info, added double checking mechanism to add the lock
-          // object so in case of concurrent query we for same block info only one lock
-          // object will be added
-          if (null == blockInfoLockObject) {
-            synchronized (blockInfoLock) {
-              // again checking the block info lock, to check whether lock object is present
-              // or not if now also not present then add a lock object
-              blockInfoLockObject = blockInfoLock.get(blockInfo);
-              if (null == blockInfoLockObject) {
-                blockInfoLockObject = new Object();
-                blockInfoLock.put(blockInfo, blockInfoLockObject);
-              }
+    List<Future<AbstractIndex>> blocksList = new ArrayList<Future<AbstractIndex>>();
+    int counter = -1;
+    for (TableBlockInfo blockInfo : tableBlocksInfos) {
+      counter++;
+      // if table block is already loaded then do not load
+      // that block
+      tableBlock = tableBlockMapTemp.get(blockInfo);
+      // if block is not loaded
+      if (null == tableBlock) {
+        // check any lock object is present in
+        // block info lock map
+        Object blockInfoLockObject = blockInfoLock.get(blockInfo);
+        // if lock object is not present then acquire
+        // the lock in block info lock and add a lock object in the map for
+        // particular block info, added double checking mechanism to add the lock
+        // object so in case of concurrent query we for same block info only one lock
+        // object will be added
+        if (null == blockInfoLockObject) {
+          synchronized (blockInfoLock) {
+            // again checking the block info lock, to check whether lock object is present
+            // or not if now also not present then add a lock object
+            blockInfoLockObject = blockInfoLock.get(blockInfo);
+            if (null == blockInfoLockObject) {
+              blockInfoLockObject = new Object();
+              blockInfoLock.put(blockInfo, blockInfoLockObject);
             }
           }
-          //acquire the lock for particular block info
-          synchronized (blockInfoLockObject) {
-            // check again whether block is present or not to avoid the
-            // same block is loaded
-            //more than once in case of concurrent query
-            tableBlock = tableBlockMapTemp.get(blockInfo);
-            // if still block is not present then load the block
-            if (null == tableBlock) {
-              // getting the data file meta data of the block
-              footer = CarbonUtil
-                  .readMetadatFile(blockInfo.getFilePath(), blockInfo.getBlockOffset(),
-                      blockInfo.getBlockLength());
-              tableBlock = new BlockIndex();
-              footer.setTableBlockInfo(blockInfo);
-              // building the block
-              tableBlock.buildIndex(Arrays.asList(footer));
-              tableBlockMapTemp.put(blockInfo, tableBlock);
-              // finally remove the lock object from block info lock as once block is loaded
-              // it will not come inside this if condition
-              blockInfoLock.remove(blockInfo);
-            }
+        }
+        //acquire the lock for particular block info
+        synchronized (blockInfoLockObject) {
+          // check again whether block is present or not to avoid the
+          // same block is loaded
+          //more than once in case of concurrent query
+          tableBlock = tableBlockMapTemp.get(blockInfo);
+          // if still block is not present then load the block
+          if (null == tableBlock) {
+            blocksList.add(executor.submit(new BlockLoaderThread(blockInfo, tableBlockMapTemp)));
           }
         }
-        loadedBlocksList.add(tableBlock);
+      } else {
+        // if blocks is already loaded then directly set the block at particular position
+        //so block will be present in sorted order
+        loadedBlock[counter] = tableBlock;
       }
-    } catch (CarbonUtilException e) {
-      LOGGER.error("Problem while loading the block");
+    }
+    // shutdown the executor gracefully and wait until all the task is finished
+    executor.shutdown();
+    try {
+      executor.awaitTermination(1, TimeUnit.HOURS);
+    } catch (InterruptedException e) {
       throw new IndexBuilderException(e);
     }
-    return loadedBlocksList;
+    // fill the block which were not loaded before to loaded blocks array
+    fillLoadedBlocks(loadedBlock, blocksList);
+    return Arrays.asList(loadedBlock);
+  }
+
+  /**
+   * Below method will be used to fill the loaded blocks to the array
+   * which will be used for query execution
+   *
+   * @param loadedBlockArray array of blocks which will be filled
+   * @param blocksList       blocks loaded in thread
+   * @throws IndexBuilderException in case of any failure
+   */
+  private void fillLoadedBlocks(AbstractIndex[] loadedBlockArray,
+      List<Future<AbstractIndex>> blocksList) throws IndexBuilderException {
+    int blockCounter = 0;
+    for (int i = 0; i < loadedBlockArray.length; i++) {
+      if (null == loadedBlockArray[i]) {
+        try {
+          loadedBlockArray[i] = blocksList.get(blockCounter++).get();
+        } catch (InterruptedException | ExecutionException e) {
+          throw new IndexBuilderException(e);
+        }
+      }
+
+    }
+  }
+
+  private AbstractIndex loadBlock(Map<TableBlockInfo, AbstractIndex> tableBlockMapTemp,
+      TableBlockInfo blockInfo) throws CarbonUtilException {
+    AbstractIndex tableBlock;
+    DataFileFooter footer;
+    // getting the data file meta data of the block
+    footer = CarbonUtil.readMetadatFile(blockInfo.getFilePath(), blockInfo.getBlockOffset(),
+        blockInfo.getBlockLength());
+    tableBlock = new BlockIndex();
+    footer.setTableBlockInfo(blockInfo);
+    // building the block
+    tableBlock.buildIndex(Arrays.asList(footer));
+    tableBlockMapTemp.put(blockInfo, tableBlock);
+    // finally remove the lock object from block info lock as once block is loaded
+    // it will not come inside this if condition
+    blockInfoLock.remove(blockInfo);
+    return tableBlock;
   }
 
   /**
@@ -228,4 +282,28 @@ public class BlockIndexStore {
     tableLockMap.remove(absoluteTableIdentifier);
     tableBlocksMap.remove(absoluteTableIdentifier);
   }
+
+  /**
+   * Thread class which will be used to load the blocks
+   */
+  private class BlockLoaderThread implements Callable<AbstractIndex> {
+    /**
+     * table block info to block index map
+     */
+    private Map<TableBlockInfo, AbstractIndex> tableBlockMap;
+
+    // block info
+    private TableBlockInfo blockInfo;
+
+    private BlockLoaderThread(TableBlockInfo blockInfo,
+        Map<TableBlockInfo, AbstractIndex> tableBlockMap) {
+      this.tableBlockMap = tableBlockMap;
+      this.blockInfo = blockInfo;
+    }
+
+    @Override public AbstractIndex call() throws Exception {
+      // load and return the loaded blocks
+      return loadBlock(tableBlockMap, blockInfo);
+    }
+  }
 }


[05/50] [abbrv] incubator-carbondata git commit: [Issue- CARBONDATA-7] fortify issue fixes, Fixed Explicit null dereferenced , Dereference null return value (#738)

Posted by ch...@apache.org.
[Issue- CARBONDATA-7] fortify issue fixes, Fixed Explicit null dereferenced ,Dereference null return value (#738)



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

Branch: refs/heads/master
Commit: a174f69e711f531e33d5979aa8845b1aee9de902
Parents: f17319d
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Sat Jun 25 12:52:50 2016 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sat Jun 25 12:52:50 2016 +0530

----------------------------------------------------------------------
 .../cache/dictionary/ColumnDictionaryInfo.java  |  5 +-
 .../datastore/block/SegmentProperties.java      |  6 +-
 .../store/filesystem/HDFSCarbonFile.java        |  3 +-
 .../store/filesystem/ViewFSCarbonFile.java      |  6 +-
 .../org/carbondata/core/util/CarbonUtil.java    |  3 +-
 .../core/writer/HierarchyValueWriterForCSV.java |  2 +-
 .../aggregator/impl/CustomAggregatorHelper.java |  6 +-
 .../query/carbon/executor/util/QueryUtil.java   | 13 +++-
 .../query/expression/ColumnExpression.java      |  3 +-
 .../executer/ColGroupFilterExecuterImpl.java    | 26 ++++---
 .../executer/RowLevelFilterExecuterImpl.java    | 26 ++++---
 .../filters/measurefilter/util/FilterUtil.java  | 77 +++++++++++---------
 .../query/util/DataFileFooterConverter.java     | 19 ++---
 13 files changed, 112 insertions(+), 83 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/core/src/main/java/org/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java b/core/src/main/java/org/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
index c9dff58..9f40406 100644
--- a/core/src/main/java/org/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
+++ b/core/src/main/java/org/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
@@ -193,7 +193,10 @@ public class ColumnDictionaryInfo extends AbstractColumnDictionaryInfo {
         int surrogateKey = sortedSurrogates.get(mid);
         byte[] dictionaryValue = getDictionaryBytesFromSurrogate(surrogateKey);
         int cmp = -1;
-        if (this.getDataType() != DataType.STRING) {
+        //fortify fix
+        if (null == dictionaryValue) {
+          cmp = -1;
+        } else if (this.getDataType() != DataType.STRING) {
           cmp = compareFilterKeyWithDictionaryKey(new String(dictionaryValue), filterKey,
               this.getDataType());
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java b/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
index c177feb..137468d 100644
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
@@ -235,7 +235,8 @@ public class SegmentProperties {
   /**
    * below method will fill dimension and measure detail of the block.
    *
-   * @param blockMetadata
+   * @param columnsInTable
+   * @param columnCardinality
    */
   private void fillDimensionAndMeasureDetails(List<ColumnSchema> columnsInTable,
       int[] columnCardinality) {
@@ -435,7 +436,8 @@ public class SegmentProperties {
       // then we need to add ordinal of that column as it belongs to same
       // column group
       if (!dimensions.get(index).isColumnar()
-          && dimensions.get(index).columnGroupId() == prvColumnGroupId) {
+          && dimensions.get(index).columnGroupId() == prvColumnGroupId
+          && null != currentColumnGroup) {
         currentColumnGroup.add(index);
       }
       // if column is not a columnar then new column group has come

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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 98e40b4..2df5a82 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
@@ -104,7 +104,8 @@ public class HDFSCarbonFile extends AbstractDFSCarbonFile {
 
   @Override
   public CarbonFile getParentFile() {
-    return new HDFSCarbonFile(fileStatus.getPath().getParent());
+    Path parent = fileStatus.getPath().getParent();
+    return null == parent ? null : new HDFSCarbonFile(parent);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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
index c7e4497..43c2ef9 100644
--- 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
@@ -101,9 +101,9 @@ public class ViewFSCarbonFile extends AbstractDFSCarbonFile {
     return files;
   }
 
-  @Override
-  public CarbonFile getParentFile() {
-    return new ViewFSCarbonFile(fileStatus.getPath().getParent());
+  @Override public CarbonFile getParentFile() {
+    Path parent = fileStatus.getPath().getParent();
+    return null == parent ? null : new ViewFSCarbonFile(parent);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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 2910105..58e6f42 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -1219,7 +1219,8 @@ public final class CarbonUtil {
           .getProperty(CarbonCommonConstants.CARBON_DDL_BASE_HDFS_URL);
       if (null != baseDFSUrl) {
         String dfsUrl = conf.get(FS_DEFAULT_FS);
-        if (dfsUrl.startsWith(HDFS_PREFIX) || dfsUrl.startsWith(VIEWFS_PREFIX)) {
+        if (null != dfsUrl && (dfsUrl.startsWith(HDFS_PREFIX) || dfsUrl
+            .startsWith(VIEWFS_PREFIX))) {
           baseDFSUrl = dfsUrl + baseDFSUrl;
         }
         if (baseDFSUrl.endsWith("/")) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/core/src/main/java/org/carbondata/core/writer/HierarchyValueWriterForCSV.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/writer/HierarchyValueWriterForCSV.java b/core/src/main/java/org/carbondata/core/writer/HierarchyValueWriterForCSV.java
index 43dd102..d75ac6f 100644
--- a/core/src/main/java/org/carbondata/core/writer/HierarchyValueWriterForCSV.java
+++ b/core/src/main/java/org/carbondata/core/writer/HierarchyValueWriterForCSV.java
@@ -127,7 +127,7 @@ public class HierarchyValueWriterForCSV {
       }
     });
 
-    if (listFiles.length == 0) {
+    if (null == listFiles || listFiles.length == 0) {
       counter = 0;
       return;
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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
index 0e23df1..4370dc7 100644
--- a/core/src/main/java/org/carbondata/query/aggregator/impl/CustomAggregatorHelper.java
+++ b/core/src/main/java/org/carbondata/query/aggregator/impl/CustomAggregatorHelper.java
@@ -120,8 +120,10 @@ public class CustomAggregatorHelper {
       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);
+        if (null != filesArray) {
+          for (int i = 0; i < filesArray.length; i++) {
+            readLevelFileAndUpdateCache(filesArray[i], tableName + '_' + columnName);
+          }
         }
       }
     } catch (IOException e) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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..a8eaa49 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
@@ -161,7 +161,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
    */
@@ -263,7 +262,7 @@ public class QueryUtil {
    * @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
+   * @param customAggExpression        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
@@ -352,7 +351,8 @@ public class QueryUtil {
    * @return
    */
   private static List<DictionaryColumnUniqueIdentifier> getDictionaryColumnUniqueIdentifierList(
-      List<String> dictionaryColumnIdList, CarbonTableIdentifier carbonTableIdentifier) {
+      List<String> dictionaryColumnIdList, CarbonTableIdentifier carbonTableIdentifier)
+      throws QueryExecutionException {
     CarbonTable carbonTable =
         CarbonMetadata.getInstance().getCarbonTable(carbonTableIdentifier.getTableUniqueName());
     List<DictionaryColumnUniqueIdentifier> dictionaryColumnUniqueIdentifiers =
@@ -360,6 +360,10 @@ public class QueryUtil {
     for (String columnIdentifier : dictionaryColumnIdList) {
       CarbonDimension dimension = CarbonMetadata.getInstance()
           .getCarbonDimensionBasedOnColIdentifier(carbonTable, columnIdentifier);
+      if (null == dimension) {
+        throw new QueryExecutionException(
+            "The column id " + columnIdentifier + " could not be resolved.");
+      }
       DictionaryColumnUniqueIdentifier dictionaryColumnUniqueIdentifier =
           new DictionaryColumnUniqueIdentifier(carbonTableIdentifier, columnIdentifier,
               dimension.getDataType());
@@ -628,7 +632,8 @@ public class QueryUtil {
       // 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) {
+          && dimensions.get(index).getDimension().columnGroupId() == prvColumnGroupId
+          && null != currentColumnGroup) {
         currentColumnGroup.add(dimensions.get(index).getDimension().getOrdinal());
       }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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
index aa0de43..8511621 100644
--- a/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
+++ b/core/src/main/java/org/carbondata/query/expression/ColumnExpression.java
@@ -89,7 +89,8 @@ public class ColumnExpression extends LeafExpression {
   }
 
   @Override public ExpressionResult evaluate(RowIntf value) {
-    ExpressionResult expressionResult = new ExpressionResult(dataType, value.getVal(colIndex));
+    ExpressionResult expressionResult =
+        new ExpressionResult(dataType, (null == value ? null : value.getVal(colIndex)));
     return expressionResult;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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
index 21847d0..e4e3812 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
@@ -145,7 +145,7 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
   /**
    * It extract min and max data for given column from stored min max value
    *
-   * @param cols
+   * @param colGrpColumns
    * @param minMaxData
    * @param columnIndex
    * @return
@@ -153,18 +153,20 @@ public class ColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
   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;
+    if (null != colGrpColumns) {
+      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;
       }
-      startIndex += colSize;
     }
     byte[] data = new byte[endIndex - startIndex];
     System.arraycopy(minMaxData, startIndex, data, 0, data.length);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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 31794a4..095f757 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
@@ -215,18 +215,20 @@ public class RowLevelFilterExecuterImpl implements FilterExecuter {
           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();
+          if (null != aggregator) {
+            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;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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
index 3903aee..74cb4cd 100644
--- 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
@@ -107,42 +107,50 @@ public final class FilterUtil {
   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());
+    if (null != filterExecuterType) {
+      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());
 
+      }
     }
+    return new RowLevelFilterExecuterImpl(
+        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getDimColEvaluatorInfoList(),
+        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getMsrColEvalutorInfoList(),
+        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getFilterExpresion(),
+        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier());
 
   }
 
@@ -1218,6 +1226,7 @@ public final class FilterUtil {
   /**
    * 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

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a174f69e/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
index d79ea5d..7ca9a3f 100644
--- a/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/carbondata/query/util/DataFileFooterConverter.java
@@ -183,15 +183,16 @@ public class DataFileFooterConverter {
     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));
+    if (null != column_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);


[45/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
index 2f394f7,0000000..cafba7d
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
@@@ -1,1235 -1,0 +1,1304 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.FilterIllegalMemberException;
 +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.ExcludeColGroupFilterExecuterImpl;
 +import org.carbondata.scan.filter.executer.ExcludeFilterExecuterImpl;
 +import org.carbondata.scan.filter.executer.FilterExecuter;
++import org.carbondata.scan.filter.executer.IncludeColGroupFilterExecuterImpl;
 +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());
++    if (null != filterExecuterType) {
++      switch (filterExecuterType) {
++        case INCLUDE:
++          return getIncludeFilterExecuter(
++              filterExpressionResolverTree.getDimColResolvedFilterInfo(), segmentProperties);
++        case EXCLUDE:
++          return getExcludeFilterExecuter(
++              filterExpressionResolverTree.getDimColResolvedFilterInfo(), segmentProperties);
++        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,
++                  segmentProperties);
++        case ROWLEVEL:
++        default:
++          return new RowLevelFilterExecuterImpl(
++              ((RowLevelFilterResolverImpl) filterExpressionResolverTree)
++                  .getDimColEvaluatorInfoList(),
++              ((RowLevelFilterResolverImpl) filterExpressionResolverTree)
++                  .getMsrColEvalutorInfoList(),
++              ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getFilterExpresion(),
++              ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
++              segmentProperties);
 +
++      }
 +    }
++    return new RowLevelFilterExecuterImpl(
++        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getDimColEvaluatorInfoList(),
++        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getMsrColEvalutorInfoList(),
++        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getFilterExpresion(),
++        ((RowLevelFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
++        segmentProperties);
 +
 +  }
 +
 +  /**
 +   * 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);
++      return new IncludeColGroupFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
 +    }
 +  }
 +
 +  /**
++   * It gives filter executer based on columnar or column group
++   *
++   * @param dimColResolvedFilterInfo
++   * @param segmentProperties
++   * @return
++   */
++  private static FilterExecuter getExcludeFilterExecuter(
++      DimColumnResolvedFilterInfo dimColResolvedFilterInfo, SegmentProperties segmentProperties) {
++
++    if (dimColResolvedFilterInfo.getDimension().isColumnar()) {
++      return new ExcludeFilterExecuterImpl(dimColResolvedFilterInfo, segmentProperties);
++    } else {
++      return new ExcludeColGroupFilterExecuterImpl(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 {
++      throws QueryExecutionException {
 +    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 FilterUnsupportedException
 +   * @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());
++        } catch (FilterIllegalMemberException e) {
++          LOGGER.debug(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
++   * @throws FilterUnsupportedException
 +   */
 +  public static DimColumnFilterInfo getFilterList(AbsoluteTableIdentifier tableIdentifier,
 +      Expression expression, ColumnExpression columnExpression, boolean isIncludeFilter)
-       throws QueryExecutionException {
++      throws QueryExecutionException, FilterUnsupportedException {
 +    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) {
++    } catch (FilterIllegalMemberException 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
++   * @throws FilterUnsupportedException
 +   */
 +  public static DimColumnFilterInfo getFilterListForRS(Expression expression,
-       ColumnExpression columnExpression, String defaultValues, int defaultSurrogate) {
++      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate)
++      throws FilterUnsupportedException {
 +    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) {
++    } catch (FilterIllegalMemberException 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
++   * @throws FilterUnsupportedException
 +   */
 +  public static DimColumnFilterInfo getFilterListForAllMembersRS(Expression expression,
 +      ColumnExpression columnExpression, String defaultValues, int defaultSurrogate,
-       boolean isIncludeFilter) {
++      boolean isIncludeFilter) throws FilterUnsupportedException {
 +    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) {
++    } catch (FilterIllegalMemberException 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()][]);
 +
 +  }
 +
 +  /**
++   * The method is used to get the single dictionary key's mask key
++   * @param surrogate
++   * @param carbonDimension
++   * @param blockLevelKeyGenerator
++   * @return
++   */
++  public static byte[] getMaskKey(int surrogate, CarbonDimension carbonDimension,
++      KeyGenerator blockLevelKeyGenerator) {
++
++    int[] keys = new int[blockLevelKeyGenerator.getDimCount()];
++    byte[] maskedKey = null;
++    Arrays.fill(keys, 0);
++    int[] rangesForMaskedByte =
++        getRangesForMaskedByte((carbonDimension.getKeyOrdinal()), blockLevelKeyGenerator);
++    try {
++      keys[carbonDimension.getKeyOrdinal()] = surrogate;
++      maskedKey = getMaskedKey(rangesForMaskedByte, blockLevelKeyGenerator.generateKey(keys));
++    } catch (KeyGenException e) {
++      LOGGER.error(e.getMessage());
++    }
++    return maskedKey;
++  }
++
++  /**
 +   * 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());
++            carbonDimension.getColumnIdentifier(), 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 SHORT:
 +          return Short.compare((Short.parseShort(dictionaryVal)), (Short.parseShort(memberVal)));
 +        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 ShortType:
 +        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;
 +  }
 +
++  /**
++   * This method will print the error log.
++   *
++   * @param e
++   */
++  public static void logError(Throwable e, boolean invalidRowsPresent) {
++    if (!invalidRowsPresent) {
++      invalidRowsPresent=true;
++      LOGGER.error(e, CarbonCommonConstants.FILTER_INVALID_MEMBER + e.getMessage());
++    }
++  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
index 0000000,0000000..8c57c14
new file mode 100644
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeColGroupFilterExecuterImpl.java
@@@ -1,0 -1,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.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.keygenerator.KeyGenerator;
++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 ExcludeColGroupFilterExecuterImpl extends ExcludeFilterExecuterImpl {
++
++  /**
++   * LOGGER
++   */
++  private static final LogService LOGGER =
++      LogServiceFactory.getLogService(ExcludeColGroupFilterExecuterImpl.class.getName());
++
++  /**
++   * @param dimColResolvedFilterInfo
++   * @param segmentProperties
++   */
++  public ExcludeColGroupFilterExecuterImpl(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);
++    bitSet.flip(0, 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.flip(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 {
++    int colGrpId = getColumnGroupId(dimColEvaluatorInfo.getColumnIndex());
++    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
++    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
++    mdKeyOrdinal.add(getMdkeyOrdinal(dimColEvaluatorInfo.getColumnIndex(), colGrpId));
++    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
++    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
++    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
++    KeyStructureInfo restructureInfos = new KeyStructureInfo();
++    restructureInfos.setKeyGenerator(keyGenerator);
++    restructureInfos.setMaskByteRanges(maskByteRanges);
++    restructureInfos.setMaxKey(maxKey);
++    restructureInfos.setMaskedBytes(maksedByte);
++    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);
++    bitSet.flip(0, 1);
++    return bitSet;
++  }
++
++  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
++    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
++  }
++
++  private int getColumnGroupId(int ordinal) {
++    int[][] columnGroups = segmentProperties.getColumnGroups();
++    int colGrpId = -1;
++    for (int i = 0; i < columnGroups.length; i++) {
++      if (columnGroups[i].length > 1) {
++        colGrpId++;
++        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
++          break;
++        }
++      }
++    }
++    return colGrpId;
++  }
++
++  public KeyGenerator getKeyGenerator(int colGrpId) {
++    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
++  }
++}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java
index 6eb7d9a,0000000..2bc5450
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/ExcludeFilterExecuterImpl.java
@@@ -1,205 -1,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.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.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;
-   }
++  protected DimColumnResolvedFilterInfo dimColEvaluatorInfo;
++  protected DimColumnExecuterFilterInfo dimColumnExecuterInfo;
++  protected SegmentProperties segmentProperties;
 +
 +  public ExcludeFilterExecuterImpl(DimColumnResolvedFilterInfo dimColEvaluatorInfo,
-       KeyGenerator blockKeyGenerator) {
-     this(dimColEvaluatorInfo);
++      SegmentProperties segmentProperties) {
++    this.dimColEvaluatorInfo = dimColEvaluatorInfo;
 +    dimColumnExecuterInfo = new DimColumnExecuterFilterInfo();
-     FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(), blockKeyGenerator,
-         dimColEvaluatorInfo.getDimension(), dimColumnExecuterInfo);
++    this.segmentProperties = segmentProperties;
++    FilterUtil.prepareKeysFromSurrogates(dimColEvaluatorInfo.getFilterValues(),
++        segmentProperties.getDimensionKeyGenerator(), dimColEvaluatorInfo.getDimension(),
++        dimColumnExecuterInfo);
 +  }
 +
 +  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder) {
-     if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
-       blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
-           dimColEvaluatorInfo.getColumnIndex());
++    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
++        .get(dimColEvaluatorInfo.getColumnIndex());
++    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
++      blockChunkHolder.getDataBlock()
++          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
 +    }
-     if (null == blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()]) {
-       blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()] =
-           blockChunkHolder.getDataBlock().getDimensionChunk(blockChunkHolder.getFileReader(),
-               dimColEvaluatorInfo.getColumnIndex());
++    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
++      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
++          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
 +    }
 +    return getFilteredIndexes(
-         blockChunkHolder.getDimensionDataChunk()[dimColEvaluatorInfo.getColumnIndex()],
++        blockChunkHolder.getDimensionDataChunk()[blockIndex],
 +        blockChunkHolder.getDataBlock().nodeSize());
 +  }
 +
-   private BitSet getFilteredIndexes(DimensionColumnDataChunk dimColumnDataChunk, int numerOfRows) {
++  protected 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) {
++              filterValues[i], false);
++      if (startKey < 0) {
 +        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) {
++              filterValues[k], false);
++      if (startKey < 0) {
 +        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/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
index 0000000,0000000..a6d8b7d
new file mode 100644
--- /dev/null
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/IncludeColGroupFilterExecuterImpl.java
@@@ -1,0 -1,0 +1,209 @@@
++/*
++ * Licensed to the Apache Software Foundation (ASF) under one
++ * or more contributor license agreements.  See the NOTICE file
++ * distributed with this work for additional information
++ * regarding copyright ownership.  The ASF licenses this file
++ * to you under the Apache License, Version 2.0 (the
++ * "License"); you may not use this file except in compliance
++ * with the License.  You may obtain a copy of the License at
++ *
++ *    http://www.apache.org/licenses/LICENSE-2.0
++ *
++ * Unless required by applicable law or agreed to in writing,
++ * software distributed under the License is distributed on an
++ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.keygenerator.KeyGenerator;
++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 IncludeColGroupFilterExecuterImpl extends IncludeFilterExecuterImpl {
++
++  /**
++   * LOGGER
++   */
++  private static final LogService LOGGER =
++      LogServiceFactory.getLogService(IncludeColGroupFilterExecuterImpl.class.getName());
++
++  /**
++   * @param dimColResolvedFilterInfo
++   * @param segmentProperties
++   */
++  public IncludeColGroupFilterExecuterImpl(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 {
++    int colGrpId = getColumnGroupId(dimColumnEvaluatorInfo.getColumnIndex());
++    KeyGenerator keyGenerator = segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
++    List<Integer> mdKeyOrdinal = new ArrayList<Integer>();
++    mdKeyOrdinal.add(getMdkeyOrdinal(dimColumnEvaluatorInfo.getColumnIndex(), colGrpId));
++    int[] maskByteRanges = QueryUtil.getMaskedByteRangeBasedOrdinal(mdKeyOrdinal, keyGenerator);
++    byte[] maxKey = QueryUtil.getMaxKeyBasedOnOrinal(mdKeyOrdinal, keyGenerator);
++    int[] maksedByte = QueryUtil.getMaskedByte(keyGenerator.getKeySizeInBytes(), maskByteRanges);
++    KeyStructureInfo restructureInfos = new KeyStructureInfo();
++    restructureInfos.setKeyGenerator(keyGenerator);
++    restructureInfos.setMaskByteRanges(maskByteRanges);
++    restructureInfos.setMaxKey(maxKey);
++    restructureInfos.setMaskedBytes(maksedByte);
++    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 colGrpColumns
++   * @param minMaxData
++   * @param columnIndex
++   * @return
++   */
++  private byte[] getMinMaxData(int[] colGrpColumns, byte[] minMaxData, int columnIndex) {
++    int startIndex = 0;
++    int endIndex = 0;
++    if (null != colGrpColumns) {
++      for (int i = 0; i < colGrpColumns.length; i++) {
++        int colGrpId = getColumnGroupId(colGrpColumns[i]);
++        int mdKeyOrdinal = getMdkeyOrdinal(colGrpColumns[i], colGrpId);
++        int[] byteRange = getKeyGenerator(colGrpId).getKeyByteOffsets(mdKeyOrdinal);
++        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;
++  }
++
++  private int getMdkeyOrdinal(int ordinal, int colGrpId) {
++    return segmentProperties.getColumnGroupMdKeyOrdinal(colGrpId, ordinal);
++  }
++
++  private int getColumnGroupId(int ordinal) {
++    int[][] columnGroups = segmentProperties.getColumnGroups();
++    int colGrpId = -1;
++    for (int i = 0; i < columnGroups.length; i++) {
++      if (columnGroups[i].length > 1) {
++        colGrpId++;
++        if (QueryUtil.searchInArray(columnGroups[i], ordinal)) {
++          break;
++        }
++      }
++    }
++    return colGrpId;
++  }
++
++  public KeyGenerator getKeyGenerator(int colGrpId) {
++    return segmentProperties.getColumnGroupAndItsKeygenartor().get(colGrpId);
++  }
++}


[19/50] [abbrv] incubator-carbondata git commit: [Bug]skip escape char in kettle and fix quote in middle of value (#751)

Posted by ch...@apache.org.
[Bug]skip escape char in kettle and fix quote in middle of value (#751)

In currectly flow,escape character will no skip in kettle,while dictionary generation will skip the escape character .and if the value no start with quote,it will no skip also.
like "ab\c"->"abc",a"b\c"->a"b\c"

also,this pr fix the value like ab"c"d,which contain double quote in middle of value.

as suggesoin,we need to make the same parser in phase 1:global dictionary and phase 2:data loading.
using same parser:univocity parser. then all defects about different parser will no comes any more.

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

Branch: refs/heads/master
Commit: 5045d73942204eddf801e0f944a1f6897f61b12e
Parents: f16eb54
Author: linyixin <li...@huawei.com>
Authored: Mon Jun 27 17:45:59 2016 +0800
Committer: Vimal-Das <vi...@gmail.com>
Committed: Mon Jun 27 02:45:59 2016 -0700

----------------------------------------------------------------------
 .../hadoop/test/util/StoreCreator.java          |  3 +-
 .../carbondata/spark/load/CarbonLoadModel.java  |  2 +
 .../carbondata/spark/load/CarbonLoaderUtil.java |  2 +
 .../execution/command/carbonTableSchema.scala   |  2 +-
 .../test/resources/datawithescapecharacter.csv  | 22 ++++++
 .../test/resources/datawithspecialcharacter.csv | 37 +++++++++
 .../dataload/TestLoadDataWithHiveSyntax.scala   | 55 +++++++++++++-
 .../api/dataloader/DataLoadModel.java           |  9 +++
 .../processing/csvload/DataGraphExecuter.java   |  8 ++
 .../csvreaderstep/BlockDataHandler.java         | 79 +++++++++++++-------
 .../processing/csvreaderstep/CsvInput.java      |  3 +-
 .../processing/csvreaderstep/CsvInputData.java  |  1 +
 .../processing/csvreaderstep/CsvInputMeta.java  | 21 +++++-
 .../dataprocessor/DataProcessTaskStatus.java    | 10 +++
 .../dataprocessor/IDataProcessStatus.java       |  2 +
 .../graphgenerator/GraphGenerator.java          |  3 +
 16 files changed, 224 insertions(+), 35 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/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..556fc8d 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
@@ -365,7 +365,7 @@ public class StoreCreator {
         0, new File(loadModel.getFactFilePath()).length());
     GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails });
     schmaModel.setBlocksID("qwqwq");
-
+    schmaModel.setEscapeCharacter("\\");
     info.setSchemaName(schemaName);
     info.setCubeName(cubeName);
 
@@ -473,6 +473,7 @@ public class StoreCreator {
     model.setTaskNo("1");
     model.setBlocksID(schmaModel.getBlocksID());
     model.setFactTimeStamp(readCurrentTime());
+    model.setEscapeCharacter(schmaModel.getEscapeCharacter());
     if (null != loadMetadataDetails && !loadMetadataDetails.isEmpty()) {
       model.setLoadNames(
           CarbonDataProcessorUtil.getLoadNameFromLoadMetaDataDetails(loadMetadataDetails));

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoadModel.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoadModel.java b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoadModel.java
index c422c9f..7125818 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoadModel.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoadModel.java
@@ -264,6 +264,7 @@ public class CarbonLoadModel implements Serializable {
     copy.taskNo = taskNo;
     copy.factTimeStamp = factTimeStamp;
     copy.segmentId = segmentId;
+    copy.escapeChar = escapeChar;
     return copy;
   }
 
@@ -300,6 +301,7 @@ public class CarbonLoadModel implements Serializable {
     copyObj.taskNo = taskNo;
     copyObj.factTimeStamp = factTimeStamp;
     copyObj.segmentId = segmentId;
+    copyObj.escapeChar = escapeChar;
     return copyObj;
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/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 0fb2621..c5eb971 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
@@ -104,6 +104,7 @@ public final class CarbonLoaderUtil {
           .getModificationOrDeletionTimesFromLoadMetadataDetails(loadMetadataDetails));
     }
     model.setBlocksID(schmaModel.getBlocksID());
+    model.setEscapeCharacter(schmaModel.getEscapeCharacter());
     model.setTaskNo(loadModel.getTaskNo());
     model.setFactTimeStamp(loadModel.getFactTimeStamp());
     boolean hdfsReadMode =
@@ -159,6 +160,7 @@ public final class CarbonLoaderUtil {
     }
 
     schmaModel.setBlocksID(loadModel.getBlocksID());
+    schmaModel.setEscapeCharacter(loadModel.getEscapeChar());
     SchemaInfo info = new SchemaInfo();
 
     info.setSchemaName(databaseName);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/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 99cb9ca..a65b7b7 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
@@ -1496,7 +1496,7 @@ private[sql] case class LoadCube(
       val delimiter = partionValues.getOrElse("delimiter", ",")
       val quoteChar = partionValues.getOrElse("quotechar", "\"")
       val fileHeader = partionValues.getOrElse("fileheader", "")
-      val escapeChar = partionValues.getOrElse("escapechar", "")
+      val escapeChar = partionValues.getOrElse("escapechar", "\\")
       val complex_delimiter_level_1 = partionValues.getOrElse("complex_delimiter_level_1", "\\$")
       val complex_delimiter_level_2 = partionValues.getOrElse("complex_delimiter_level_2", "\\:")
       val multiLine = partionValues.getOrElse("multiline", "false").trim.toLowerCase match {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/integration/spark/src/test/resources/datawithescapecharacter.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datawithescapecharacter.csv b/integration/spark/src/test/resources/datawithescapecharacter.csv
new file mode 100644
index 0000000..9bda54d
--- /dev/null
+++ b/integration/spark/src/test/resources/datawithescapecharacter.csv
@@ -0,0 +1,22 @@
+imei,specialchar
+1AA1,hash#124
+1AA2,space 125
+1AA3,ampersand&&hi
+1AA4,escape\\esc
+1AA44,"escape\esc"
+1AA5,not!hi
+1AA6,braces(hi)
+1AA7,percentage%hi
+1AA8,Tilde~~
+1AA9,dollar$hi
+1AA10,star***hi
+1AA11,colon:hi
+1AA12,semi;colon
+1AA13,quote'1'22
+1AA14,underscore_hi
+1AA15,equals=hi
+1AA16,plus+hi
+1232,"ayush@b.com"
+12323,"ayush@@b.com"
+12345,"\u897f\u5b89\\u54b8\u9633"
+12346,"\u897f\u5b89\\\u54b8\u9633"

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/integration/spark/src/test/resources/datawithspecialcharacter.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datawithspecialcharacter.csv b/integration/spark/src/test/resources/datawithspecialcharacter.csv
new file mode 100644
index 0000000..f807120
--- /dev/null
+++ b/integration/spark/src/test/resources/datawithspecialcharacter.csv
@@ -0,0 +1,37 @@
+imei,specialchar
+1AA0,double"specialchar"quote
+1AA1,hash#124
+1AA2,space 125
+1AA3,ampersand&&hi
+1AA4,escape\esc
+1AA5,not!hi
+1AA6,braces(hi)
+1AA7,percentage%hi
+1AA8,Tilde~~
+1AA9,dollar$hi
+1AA10,star***hi
+1AA11,colon:hi
+1AA12,semi;colon
+1AA13,quote'1'22
+1AA14,underscore_hi
+1AA15,equals=hi
+1AA16,plus+hi
+1AA17,minus-hi
+1AA18,combination ~!@#$%^&*()_+| CA
+1AA19,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC
+1AA20,LonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmember
+1AA21,char123234Number
+1AA22,\\  esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF
+1AA23,select * from sale_fact
+1AA24,pipe|hi
+1AA25,Curly{braces}hi
+1AA26,braces[hi]
+1AA27,question?hi
+1AA28,lessthan< hi
+1AA29,morethan>hi
+1AA30,dot.hi
+1AA31,comma,hi
+1AA32,escapecomma\,hi
+1AA33,fslash/hi
+1AA34,a@b
+1AA35,\\n

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
index 6765b2a..a75f020 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
@@ -202,7 +202,7 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
     sql("drop table LowErcasEcube")
   }
   
-  test("test carbon table data loading using escape char") {
+  test("test carbon table data loading using escape char 1") {
     sql("DROP TABLE IF EXISTS t3")
 
     sql("""
@@ -222,7 +222,58 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
     sql("DROP TABLE IF EXISTS t3")
   }
-  
+
+  test("test carbon table data loading using escape char 2") {
+    sql("DROP TABLE IF EXISTS t3")
+
+    sql("""
+         CREATE TABLE t3(imei string,specialchar string)
+         STORED BY 'org.apache.carbondata.format'
+        """)
+
+    sql("""
+       LOAD DATA LOCAL INPATH './src/test/resources/datawithescapecharacter.csv' into table t3
+          options ('DELIMITER'=',', 'QUOTECHAR'='\"','ESCAPECHAR'='\')
+        """)
+    checkAnswer(sql("select count(*) from t3"), Seq(Row(21)))
+    checkAnswer(sql("select specialchar from t3 where imei = '1AA44'"),Seq(Row("escapeesc")))
+    sql("DROP TABLE IF EXISTS t3")
+  }
+
+  test("test carbon table data loading using escape char 3") {
+    sql("DROP TABLE IF EXISTS t3")
+
+    sql("""
+         CREATE TABLE t3(imei string,specialchar string)
+         STORED BY 'org.apache.carbondata.format'
+        """)
+
+    sql("""
+       LOAD DATA LOCAL INPATH './src/test/resources/datawithescapecharacter.csv' into table t3
+          options ('DELIMITER'=',', 'QUOTECHAR'='\"','ESCAPECHAR'='@')
+        """)
+    checkAnswer(sql("select count(*) from t3"), Seq(Row(21)))
+    checkAnswer(sql("select specialchar from t3 where imei in ('1232','12323')"),Seq(Row
+    ("ayush@b.com"),Row("ayushb.com")))
+    sql("DROP TABLE IF EXISTS t3")
+  }
+  test("test carbon table data loading with special character") {
+    sql("DROP TABLE IF EXISTS t3")
+
+    sql("""
+         CREATE TABLE t3(imei string,specialchar string)
+         STORED BY 'org.apache.carbondata.format'
+        """)
+
+    sql("""
+       LOAD DATA LOCAL INPATH './src/test/resources/datawithspecialcharacter.csv' into table t3
+          options ('DELIMITER'=',', 'QUOTECHAR'='\"')
+       """)
+    checkAnswer(sql("select count(*) from t3"), Seq(Row(36)))
+    sql("DROP TABLE IF EXISTS t3")
+  }
+
+
   override def afterAll {
     sql("drop table carbontable")
     sql("drop table hivetable")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/processing/src/main/java/org/carbondata/processing/api/dataloader/DataLoadModel.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/api/dataloader/DataLoadModel.java b/processing/src/main/java/org/carbondata/processing/api/dataloader/DataLoadModel.java
index 0236c80..dd545c9 100644
--- a/processing/src/main/java/org/carbondata/processing/api/dataloader/DataLoadModel.java
+++ b/processing/src/main/java/org/carbondata/processing/api/dataloader/DataLoadModel.java
@@ -59,6 +59,7 @@ public class DataLoadModel {
    */
   private String factTimeStamp;
 
+  private String escapeCharacter;
   /**
    * @return Returns the schemaInfo.
    */
@@ -188,5 +189,13 @@ public class DataLoadModel {
   public void setFactTimeStamp(String factTimeStamp) {
     this.factTimeStamp = factTimeStamp;
   }
+
+  public String getEscapeCharacter() {
+    return escapeCharacter;
+  }
+
+  public void setEscapeCharacter(String escapeCharacter) {
+    this.escapeCharacter = escapeCharacter;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/processing/src/main/java/org/carbondata/processing/csvload/DataGraphExecuter.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/csvload/DataGraphExecuter.java b/processing/src/main/java/org/carbondata/processing/csvload/DataGraphExecuter.java
index 6526e90..ca21ec1 100644
--- a/processing/src/main/java/org/carbondata/processing/csvload/DataGraphExecuter.java
+++ b/processing/src/main/java/org/carbondata/processing/csvload/DataGraphExecuter.java
@@ -327,6 +327,8 @@ public class DataGraphExecuter {
                 .getTextInputFiles(model.getCsvHeader(), builder, measuresInCSVFile, ",");
             ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputParams);
             ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
+            ((CsvInputMeta) step.getStepMetaInterface())
+              .setEscapeCharacter(model.getEscapeCharacter());
             ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(false);
 
           } else if (model.getFilesToProcess().size() > 0) {
@@ -337,6 +339,8 @@ public class DataGraphExecuter {
                     model.getCsvDelimiter());
             ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
             ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
+            ((CsvInputMeta) step.getStepMetaInterface())
+              .setEscapeCharacter(model.getEscapeCharacter());
             ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(true);
           }
         }
@@ -415,6 +419,8 @@ public class DataGraphExecuter {
                 .getTextInputFiles(model.getCsvHeader(), builder, measuresInCSVFile, ",");
             ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
             ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
+            ((CsvInputMeta) step.getStepMetaInterface())
+                .setEscapeCharacter(model.getEscapeCharacter());
             ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(false);
 
           } else if (model.getFilesToProcess().size() > 0) {
@@ -425,6 +431,8 @@ public class DataGraphExecuter {
                     model.getCsvDelimiter());
             ((CsvInputMeta) step.getStepMetaInterface()).setInputFields(inputFields);
             ((CsvInputMeta) step.getStepMetaInterface()).setDelimiter(model.getCsvDelimiter());
+            ((CsvInputMeta) step.getStepMetaInterface())
+              .setEscapeCharacter(model.getEscapeCharacter());
             ((CsvInputMeta) step.getStepMetaInterface()).setHeaderPresent(true);
           }
         }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/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 29b4a54..4a56b85 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
@@ -136,29 +136,24 @@ public class BlockDataHandler {
     return false;
   }
 
-  /**
-   * <pre>
-   * [abcd "" defg] --> [abcd " defg]
-   * [""""] --> [""]
-   * [""] --> ["]
-   * </pre>
-   *
-   * @return the byte array with escaped enclosures escaped.
-   */
-  public byte[] removeEscapedEnclosures(byte[] field, int nrEnclosuresFound) {
-    byte[] result = new byte[field.length - nrEnclosuresFound];
+  public byte[] removeEscapeChar(byte[] field, byte[] escapeChar) {
+    byte[] result = new byte[field.length];
     int resultIndex = 0;
     for (int i = 0; i < field.length; i++) {
-      if (field[i] == data.enclosure[0]) {
-        if (!(i + 1 < field.length && field[i + 1] == data.enclosure[0])) {
-          // Not an escaped enclosure...
+      if (field[i] != escapeChar[0]) {
+        result[resultIndex++] = field[i];
+      }
+      if (i + 1 < field.length) {
+        if (field[i] == escapeChar[0] && field[i + 1] == escapeChar[0]) {
           result[resultIndex++] = field[i];
+          i++;
         }
-      } else {
-        result[resultIndex++] = field[i];
       }
     }
-    return result;
+
+    byte[] finalResult = new byte[resultIndex];
+    System.arraycopy(result,0,finalResult,0,resultIndex);
+    return finalResult;
   }
 
   protected boolean openFile(StepMetaInterface smi, StepDataInterface sdi, TransMeta trans,
@@ -166,6 +161,7 @@ public class BlockDataHandler {
     try {
       this.meta = (CsvInputMeta) smi;
       this.data = (CsvInputData) sdi;
+      this.data.preferredBufferSize = Integer.parseInt(this.meta.getBufferSize());
       this.transMeta = trans;
       // Close the previous file...
       if (this.bufferedInputStream != null) {
@@ -300,7 +296,8 @@ public class BlockDataHandler {
         //
         boolean delimiterFound = false;
         boolean enclosureFound = false;
-        int escapedEnclosureFound = 0;
+        boolean quoteAfterDelimiter = false;
+        boolean quoteBeforeDelimiterOrCrLf = false;
         while (!delimiterFound) {
           // If we find the first char, we might find others as well ;-)
           // Single byte delimiters only for now.
@@ -379,8 +376,11 @@ public class BlockDataHandler {
           //
           else if (data.enclosure != null && data.enclosureMatcher
               .matchesPattern(this.byteBuffer, this.endBuffer, data.enclosure)) {
-
+            if(this.startBuffer == this.endBuffer){
+              quoteAfterDelimiter = true;
+            }
             enclosureFound = true;
+            boolean outOfEnclosureFlag = false;
             boolean keepGoing;
             do {
               if (this.increaseEndBuffer()) {
@@ -405,6 +405,7 @@ public class BlockDataHandler {
                   .matchesPattern(this.byteBuffer, this.endBuffer,
                       data.enclosure);
               if (!keepGoing) {
+                outOfEnclosureFlag = !outOfEnclosureFlag;
                 // We found an enclosure character.
                 // Read another byte...
                 if (this.increaseEndBuffer()) {
@@ -420,7 +421,7 @@ public class BlockDataHandler {
                     .matchesPattern(this.byteBuffer, this.endBuffer,
                         data.enclosure);
                 if (keepGoing) {
-                  escapedEnclosureFound++;
+                  outOfEnclosureFlag = !outOfEnclosureFlag;
                 } else {
                   /**
                    * <pre>
@@ -444,6 +445,26 @@ public class BlockDataHandler {
                 }
 
               }
+              if (!keepGoing) {
+                if (data.enclosureMatcher
+                    .matchesPattern(this.byteBuffer, this.endBuffer - 1, data.enclosure)) {
+                  quoteBeforeDelimiterOrCrLf = true;
+                }
+              }
+              if (outOfEnclosureFlag) {
+                keepGoing = !(data.delimiterMatcher
+                  .matchesPattern(this.byteBuffer, this.endBuffer,
+                    data.delimiter) || data.crLfMatcher
+                  .isReturn(this.byteBuffer, this.endBuffer)
+                  || data.crLfMatcher
+                  .isLineFeed(this.byteBuffer, this.endBuffer));
+              }
+
+              if (quoteBeforeDelimiterOrCrLf && quoteAfterDelimiter) {
+                enclosureFound = true;
+              } else {
+                enclosureFound = false;
+              }
             } while (keepGoing);
 
             // Did we reach the end of the buffer?
@@ -480,15 +501,14 @@ public class BlockDataHandler {
         // data.byteBuffer[data.startBuffer]
         //
         int length =
-            calculateFieldLength(newLineFound, newLines, enclosureFound, endOfBuffer);
+            calculateFieldLength(newLineFound, newLines, enclosureFound, endOfBuffer,
+              quoteAfterDelimiter, quoteBeforeDelimiterOrCrLf);
 
         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);
+        if(quoteAfterDelimiter && quoteBeforeDelimiterOrCrLf){
+          field = removeEscapeChar(field,data.escapeCharacter);
         }
 
         if (doConversions) {
@@ -619,7 +639,7 @@ public class BlockDataHandler {
   }
 
   private int calculateFieldLength(boolean newLineFound, int newLines, boolean enclosureFound,
-      boolean endOfBuffer) {
+      boolean endOfBuffer, boolean quoteAfterDelimeter, boolean quoteBeforeDelimeterOrCrLf) {
 
     int length = this.endBuffer - this.startBuffer;
     if (newLineFound) {
@@ -631,6 +651,7 @@ public class BlockDataHandler {
         this.startBuffer++; // offset for the enclosure in last field before EOF
       }
     }
+
     if (enclosureFound) {
       this.startBuffer++;
       length -= 2;
@@ -638,6 +659,12 @@ public class BlockDataHandler {
         length = 0;
       }
     }
+
+    if (!endOfBuffer && (quoteAfterDelimeter && !quoteBeforeDelimeterOrCrLf)) {
+      this.startBuffer++;
+      length--;
+    }
+
     if (length <= 0) {
       length = 0;
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/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 76d5716..42f20cc 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
@@ -522,7 +522,8 @@ public class CsvInput extends BaseStep implements StepInterface {
       try {
         data.delimiter = data.encodingType
             .getBytes(environmentSubstitute(meta.getDelimiter()), meta.getEncoding());
-
+        data.escapeCharacter = data.encodingType
+          .getBytes(environmentSubstitute(meta.getEscapeCharacter()), meta.getEncoding());
         if (Const.isEmpty(meta.getEnclosure())) {
           data.enclosure = null;
         } else {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
index 77eaf1b..f97afed 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
@@ -30,6 +30,7 @@ public class CsvInputData extends BaseStepData implements StepDataInterface {
 
   public byte[] delimiter;
   public byte[] enclosure;
+  public byte[] escapeCharacter;
   public int preferredBufferSize;
   public int totalNumberOfSteps;
   public boolean parallel;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/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 6f895b1..0e4c2e7 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputMeta.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputMeta.java
@@ -97,6 +97,8 @@ public class CsvInputMeta extends BaseStepMeta
 
   private String blocksID;
 
+  private String escapeCharacter;
+
   public CsvInputMeta() {
     super(); // allocate BaseStepMeta
     allocate(0);
@@ -116,6 +118,7 @@ public class CsvInputMeta extends BaseStepMeta
     bufferSize = "50000";
     currentRestructNumber = -1;
     blocksID = "";
+    escapeCharacter ="\\";
   }
 
   private void readData(Node stepnode) throws KettleXMLException {
@@ -152,7 +155,7 @@ public class CsvInputMeta extends BaseStepMeta
       currentRestructNumber =
           Integer.parseInt(XMLHandler.getTagValue(stepnode, "currentRestructNumber"));
       blocksID = XMLHandler.getTagValue(stepnode, "blocksID");
-
+      escapeCharacter = XMLHandler.getTagValue(stepnode, "escapeCharacter");
       Node fields = XMLHandler.getSubNode(stepnode, getXmlCode("FIELDS"));
       int nrfields = XMLHandler.countNodes(fields, getXmlCode("FIELD"));
 
@@ -169,7 +172,8 @@ public class CsvInputMeta extends BaseStepMeta
         inputFields[i].setFormat(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_FORMAT")));
         inputFields[i]
             .setCurrencySymbol(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_CURRENCY")));
-        inputFields[i].setDecimalSymbol(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_DECIMAL")));
+        inputFields[i].setDecimalSymbol(XMLHandler.getTagValue(fnode,
+            getXmlCode("FIELD_DECIMAL")));
         inputFields[i].setGroupSymbol(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_GROUP")));
         inputFields[i]
             .setLength(Const.toInt(XMLHandler.getTagValue(fnode, getXmlCode("FIELD_LENGTH")), -1));
@@ -214,7 +218,7 @@ public class CsvInputMeta extends BaseStepMeta
     retval.append("    ")
         .append(XMLHandler.addTagValue("currentRestructNumber", currentRestructNumber));
     retval.append("    ").append(XMLHandler.addTagValue("blocksID", blocksID));
-
+    retval.append("    ").append(XMLHandler.addTagValue("escapeCharacter", escapeCharacter));
     retval.append("    ").append(XMLHandler.openTag(getXmlCode("FIELDS"))).append(Const.CR);
     for (int i = 0; i < inputFields.length; i++) {
       TextFileInputField field = inputFields[i];
@@ -266,6 +270,7 @@ public class CsvInputMeta extends BaseStepMeta
       encoding = rep.getStepAttributeString(idStep, getRepCode("ENCODING"));
       currentRestructNumber = (int) rep.getStepAttributeInteger(idStep, "currentRestructNumber");
       blocksID = rep.getStepAttributeString(idStep, getRepCode("blocksID"));
+      escapeCharacter = rep.getStepAttributeString(idStep, getRepCode("escapeCharacter"));
       int nrfields = rep.countNrStepAttributes(idStep, getRepCode("FIELD_NAME"));
 
       allocate(nrfields);
@@ -320,6 +325,8 @@ public class CsvInputMeta extends BaseStepMeta
       rep.saveStepAttribute(idTransformation, idStep, "currentRestructNumber",
           currentRestructNumber);
       rep.saveStepAttribute(idTransformation, idStep, getRepCode("blocksID"), blocksID);
+      rep.saveStepAttribute(idTransformation, idStep, getRepCode("escapeCharacter"),
+          escapeCharacter);
       for (int i = 0; i < inputFields.length; i++) {
         TextFileInputField field = inputFields[i];
 
@@ -605,7 +612,11 @@ public class CsvInputMeta extends BaseStepMeta
   }
 
   public String getEscapeCharacter() {
-    return null;
+    return escapeCharacter;
+  }
+
+  public void setEscapeCharacter(String escapeCharacter){
+    this.escapeCharacter = escapeCharacter;
   }
 
   public String getFileType() {
@@ -811,6 +822,8 @@ public class CsvInputMeta extends BaseStepMeta
           currentRestructNumber = (Integer) entry.getValue();
         } else if ("blocksID".equals(attributeKey)) {
           blocksID = (String) entry.getValue();
+        } else if ("escapeCharacter".equals(attributeKey)) {
+          escapeCharacter = (String) entry.getValue();
         } else {
           throw new RuntimeException(
               "Unhandled metadata injection of attribute: " + attr.toString() + " - " + attr

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/processing/src/main/java/org/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/dataprocessor/DataProcessTaskStatus.java b/processing/src/main/java/org/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
index b48393d..44ce52b 100644
--- a/processing/src/main/java/org/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
+++ b/processing/src/main/java/org/carbondata/processing/dataprocessor/DataProcessTaskStatus.java
@@ -87,6 +87,8 @@ public class DataProcessTaskStatus implements IDataProcessStatus, Serializable {
 
   private String blocksID;
 
+  private String escapeCharacter;
+
   public DataProcessTaskStatus(String schemaName, String cubeName, String tableName) {
     this.schemaName = schemaName;
     this.cubeName = cubeName;
@@ -290,4 +292,12 @@ public class DataProcessTaskStatus implements IDataProcessStatus, Serializable {
   public void setBlocksID(String blocksID) {
     this.blocksID = blocksID;
   }
+
+  public String getEscapeCharacter() {
+    return escapeCharacter;
+  }
+
+  public void setEscapeCharacter(String escapeCharacter) {
+    this.escapeCharacter = escapeCharacter;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/processing/src/main/java/org/carbondata/processing/dataprocessor/IDataProcessStatus.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/dataprocessor/IDataProcessStatus.java b/processing/src/main/java/org/carbondata/processing/dataprocessor/IDataProcessStatus.java
index 23b40e8..54c7463 100644
--- a/processing/src/main/java/org/carbondata/processing/dataprocessor/IDataProcessStatus.java
+++ b/processing/src/main/java/org/carbondata/processing/dataprocessor/IDataProcessStatus.java
@@ -197,4 +197,6 @@ public interface IDataProcessStatus {
   void setCsvDelimiter(String csvDelimiter);
 
   String getBlocksID();
+
+  String getEscapeCharacter();
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/5045d739/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 00f233c..d074473 100644
--- a/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
+++ b/processing/src/main/java/org/carbondata/processing/graphgenerator/GraphGenerator.java
@@ -178,6 +178,7 @@ public class GraphGenerator {
   private String factStoreLocation;
   private int currentRestructNumber;
   private String blocksID;
+  private String escapeCharacter;
   /**
    * task id, each spark task has a unique id
    */
@@ -211,6 +212,7 @@ public class GraphGenerator {
     this.taskNo = dataLoadModel.getTaskNo();
     this.factTimeStamp = dataLoadModel.getFactTimeStamp();
     this.segmentId = segmentId;
+    this.escapeCharacter = dataLoadModel.getEscapeCharacter();
     initialise();
     LOGGER.info("************* Is Columnar Storage" + isColumnar);
   }
@@ -438,6 +440,7 @@ public class GraphGenerator {
         CarbonCommonConstants.CSV_READ_BUFFER_SIZE_DEFAULT));
     //set blocks info id
     csvInputMeta.setBlocksID(this.blocksID);
+    csvInputMeta.setEscapeCharacter(this.escapeCharacter);
     csvDataStep.setDraw(true);
     csvDataStep.setDescription("Read raw data from " + GraphGeneratorConstants.CSV_INPUT);
 


[14/50] [abbrv] incubator-carbondata git commit: [Bug] Added comments for carbon.properties.template and Carbonindex file cleanup fix (#749)

Posted by ch...@apache.org.
[Bug] Added comments for carbon.properties.template and Carbonindex file cleanup fix (#749)

* Added comments for carbon.properties.template
* clean files not cleaning .carbonindex files

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

Branch: refs/heads/master
Commit: 720e8d6152e91be7276be5a7c1317d17caf5b30b
Parents: 24c47c2
Author: nareshpr <pr...@gmail.com>
Authored: Sat Jun 25 19:53:02 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Sat Jun 25 19:53:02 2016 +0530

----------------------------------------------------------------------
 conf/carbon.properties.template                 | 42 ++++++++++----------
 .../core/carbon/path/CarbonTablePath.java       | 14 +++++++
 .../core/constants/CarbonCommonConstants.java   | 12 +++---
 .../spark/load/DeleteLoadFolders.java           |  5 ++-
 .../store/SingleThreadFinalSortFilesMerger.java |  6 +--
 5 files changed, 47 insertions(+), 32 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/720e8d61/conf/carbon.properties.template
----------------------------------------------------------------------
diff --git a/conf/carbon.properties.template b/conf/carbon.properties.template
index 314320c..8dd6598 100644
--- a/conf/carbon.properties.template
+++ b/conf/carbon.properties.template
@@ -10,13 +10,13 @@ carbon.kettle.home=$<SPARK_HOME>/carbonlib/carbonplugins
 
 #################### Performance Configuration ##################
 ######## DataLoading Configuration ########
-#File read buffer size used during sorting:MIN=:MAX=
+#File read buffer size used during sorting(in MB) :MIN=1:MAX=100
 carbon.sort.file.buffer.size=20
-#Rowset size exchanged between data load graph steps.:MIN=:MAX=
+#Rowset size exchanged between data load graph steps :MIN=500:MAX=1000000
 carbon.graph.rowset.size=100000
-#Number of cores to be used while data loading:MIN=:MAX=
+#Number of cores to be used while data loading
 carbon.number.of.cores.while.loading=6
-#CARBON sort size.:MIN=:MAX=
+#Record count to sort and write to temp intermediate files
 carbon.sort.size=500000
 #Algorithm for hashmap for hashkey calculation
 carbon.enableXXHash=true
@@ -27,43 +27,45 @@ carbon.enableXXHash=true
 #enable prefetch of data during merge sort while reading data from sort temp files in data loading
 #carbon.merge.sort.prefetch=true
 ######## Compaction Configuration ########
-#Number of cores to be used while compacting:MIN=:MAX=
+#Number of cores to be used while compacting
 carbon.number.of.cores.while.compacting=2
-#default minor compaction in MBs
-carbon.minor.compaction.size=256
-#default major compaction in MBs
+#For minor compaction, Number of segments to be merged in stage 1, number of compacted segments to be merged in stage 2.
+carbon.compaction.level.threshold=4,3
+#default size (in MB) for major compaction to be triggered
 carbon.major.compaction.size=1024
 ######## Query Configuration ########
-#Number of cores to be used.:MIN=:MAX=
+#Number of cores to be used while querying
 carbon.number.of.cores=4
-#Carbon Inmemory record size:MIN=:MAX=
-carbon.inmemory.record.size=100000
+#Number of records to be in memory while querying :MIN=100000:MAX=240000
+carbon.inmemory.record.size=120000
 #Improves the performance of filter query
 carbon.enable.quick.filter=false
+##number of core to load the blocks in driver
+#no.of.cores.to.load.blocks.in.driver=10
 
 #################### Extra Configuration ##################
 ##Timestamp format of input data used for timestamp data type.
 #carbon.timestamp.format=yyyy-MM-dd HH:mm:ss
 ######## Dataload Configuration ########
-######File write buffer size used during sorting.
+##File write buffer size used during sorting.
 #carbon.sort.file.write.buffer.size=10485760
+##Locking mechanism for data loading on a table
+#carbon.lock.type=LOCALLOCK
 ##Minimum no of intermediate files after which sort merged to be started.
 #carbon.sort.intermediate.files.limit=20
 ##space reserved in percentage for writing block meta data in carbon data file
 #carbon.block.meta.size.reserved.percentage=10
 ##csv reading buffer size.
 #carbon.csv.read.buffersize.byte=1048576
-##space reserved in percentage for writing block meta data in carbon data file
-#carbon.block.meta.size.reserved.percentage=10
-##High Cardinality value
+##To identify and apply compression for non-high cardinality columns
 #high.cardinality.value=100000
-##CARBON maximum no of threads used for sorting.
-#carbon.max.thread.for.sorting=3
+##maximum no of threads used for reading intermediate files for final merging.
+#carbon.merge.sort.reader.thread=3
 ##Carbon blocklet size. Note: this configuration cannot be change once store is generated
 #carbon.blocklet.size=120000
-##How to times retry to get the lock
+##number of retries to get the metadata lock for loading data to table
 #carbon.load.metadata.lock.retries=3
-##Maximum number of blocklets written in a single file.:Min=1:Max=1000
+##Maximum number of blocklets written in a single file :Min=1:Max=1000
 #carbon.max.file.size=100
 ##Interval between the retries to get the lock
 #carbon.load.metadata.lock.retry.timeout.sec=5
@@ -83,8 +85,6 @@ carbon.enable.quick.filter=false
 #max.query.execution.time=60
 ##Min max is feature added to enhance query performance. To disable this feature, make it false.
 #carbon.enableMinMax=true
-##number of core to load the blocks in driver
-#no.of.cores.to.load.blocks.in.driver=10
 ######## Global Dictionary Configurations ########
 ##To enable/disable identify high cardinality during first data loading
 #high.cardinality.identify.enable=true

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/720e8d61/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
index 8dcd207..5f862bc 100644
--- a/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
+++ b/core/src/main/java/org/carbondata/core/carbon/path/CarbonTablePath.java
@@ -98,6 +98,20 @@ public class CarbonTablePath extends Path {
   }
 
   /**
+   * check if it is carbon index file matching extension
+   *
+   * @param fileNameWithPath
+   * @return boolean
+   */
+  public static boolean isCarbonIndexFile(String fileNameWithPath) {
+    int pos = fileNameWithPath.lastIndexOf('.');
+    if (pos != -1) {
+      return fileNameWithPath.substring(pos).startsWith(INDEX_FILE_EXT);
+    }
+    return false;
+  }
+
+  /**
    * gets table path
    */
   public String getPath() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/720e8d61/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 a120d00..473e764 100644
--- a/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
@@ -250,7 +250,7 @@ public final class CarbonCommonConstants {
   /**
    * SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE
    */
-  public static final String SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE = "10";
+  public static final String SORT_INTERMEDIATE_FILES_LIMIT_DEFAULT_VALUE = "20";
   /**
    * MERGERD_EXTENSION
    */
@@ -349,13 +349,13 @@ public final class CarbonCommonConstants {
    */
   public static final String SORT_TEMP_FILE_EXT = ".sorttemp";
   /**
-   * CARBON_MAX_THREAD_FOR_SORTING
+   * CARBON_MERGE_SORT_READER_THREAD
    */
-  public static final String CARBON_MAX_THREAD_FOR_SORTING = "carbon.max.thread.for.sorting";
+  public static final String CARBON_MERGE_SORT_READER_THREAD = "carbon.merge.sort.reader.thread";
   /**
-   * CARBON_MAX_THREAD_FOR_SORTING
+   * CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE
    */
-  public static final String CARBON_MAX_THREAD_FOR_SORTING_DEFAULTVALUE = "2";
+  public static final String CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE = "3";
   /**
    * IS_SORT_TEMP_FILE_COMPRESSION_ENABLED
    */
@@ -645,7 +645,7 @@ public final class CarbonCommonConstants {
   /**
    * xxhash algorithm property for hashmap. Default value false
    */
-  public static final String ENABLE_XXHASH_DEFAULT = "false";
+  public static final String ENABLE_XXHASH_DEFAULT = "true";
 
   /**
    * default charset to be used for reading and writing

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/720e8d61/integration/spark/src/main/java/org/carbondata/spark/load/DeleteLoadFolders.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/load/DeleteLoadFolders.java b/integration/spark/src/main/java/org/carbondata/spark/load/DeleteLoadFolders.java
index 1bb48d4..01d785a 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/load/DeleteLoadFolders.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/load/DeleteLoadFolders.java
@@ -38,6 +38,7 @@ import java.util.List;
 import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 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;
@@ -180,8 +181,8 @@ public final class DeleteLoadFolders {
         CarbonFile[] filesToBeDeleted = file.listFiles(new CarbonFileFilter() {
 
           @Override public boolean accept(CarbonFile file) {
-            return (file.getName().endsWith(CarbonCommonConstants.FACT_FILE_EXT) || file.getName()
-                .endsWith(CarbonCommonConstants.MEASUREMETADATA_FILE_EXT));
+            return (CarbonTablePath.isCarbonDataFile(file.getName())
+                || CarbonTablePath.isCarbonIndexFile(file.getName()));
           }
         });
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/720e8d61/processing/src/main/java/org/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java b/processing/src/main/java/org/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
index 9096b07..85f7c54 100644
--- a/processing/src/main/java/org/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
+++ b/processing/src/main/java/org/carbondata/processing/store/SingleThreadFinalSortFilesMerger.java
@@ -162,11 +162,11 @@ public class SingleThreadFinalSortFilesMerger {
     int maxThreadForSorting = 0;
     try {
       maxThreadForSorting = Integer.parseInt(CarbonProperties.getInstance()
-          .getProperty(CarbonCommonConstants.CARBON_MAX_THREAD_FOR_SORTING,
-              CarbonCommonConstants.CARBON_MAX_THREAD_FOR_SORTING_DEFAULTVALUE));
+          .getProperty(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD,
+              CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE));
     } catch (NumberFormatException e) {
       maxThreadForSorting =
-          Integer.parseInt(CarbonCommonConstants.CARBON_MAX_THREAD_FOR_SORTING_DEFAULTVALUE);
+          Integer.parseInt(CarbonCommonConstants.CARBON_MERGE_SORT_READER_THREAD_DEFAULTVALUE);
     }
     ExecutorService service = Executors.newFixedThreadPool(maxThreadForSorting);
 


[02/50] [abbrv] incubator-carbondata git commit: Update README.md

Posted by ch...@apache.org.
Update README.md

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

Branch: refs/heads/master
Commit: acbacb8c3dd92dd22045477d1e041d66336358a3
Parents: b0f2f06
Author: Liang Chen <ch...@huawei.com>
Authored: Sat Jun 25 03:01:11 2016 +0530
Committer: GitHub <no...@github.com>
Committed: Sat Jun 25 03:01:11 2016 +0530

----------------------------------------------------------------------
 README.md | 8 +++++++-
 1 file changed, 7 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/acbacb8c/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index 0b87fd0..a2cd048 100644
--- a/README.md
+++ b/README.md
@@ -78,5 +78,11 @@ Read the [quick start](https://github.com/HuaweiBigData/carbondata/wiki/Quick-St
 This is an open source project for everyone, and we are always open to people who want to use this system or contribute to it. 
 This guide document introduce [how to contribute to CarbonData](https://github.com/HuaweiBigData/carbondata/wiki/How-to-contribute-and-Code-Style).
 
+### Contact us
+To get involved in CarbonData:
+
+* [Subscribe](mailto:dev-subscribe@carbondata.incubator.apache.org) then [mail](mailto:dev@carbondata.incubator.apache.org) to us
+* Report issues on [Jira](https://issues.apache.org/jira/browse/CARBONDATA).
+
 ### About
-CarbonData project original contributed from the [Huawei](http://www.huawei.com), in progress of donating this open source project to Apache Software Foundation for leveraging big data ecosystem. 
+CarbonData project original contributed from the [Huawei](http://www.huawei.com)


[33/50] [abbrv] incubator-carbondata git commit: [Bug] Reverting changes for path & tablePath in external table (#770)

Posted by ch...@apache.org.
[Bug] Reverting changes for path & tablePath in external table (#770)



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

Branch: refs/heads/master
Commit: a743b77175ac16ab37879ee1f3aeaf14376f054d
Parents: 6701eee
Author: nareshpr <pr...@gmail.com>
Authored: Tue Jun 28 06:44:11 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Tue Jun 28 06:44:11 2016 +0530

----------------------------------------------------------------------
 .../org/apache/spark/sql/execution/command/carbonTableSchema.scala | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/a743b771/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 ffc7e60..bcd6dd3 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
@@ -1265,7 +1265,7 @@ 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 "$tablePath", path "$tablePath") """)
+          s""" OPTIONS (cubename "$dbName.$tbName", tablePath "$tablePath") """)
               .collect
       } catch {
         case e: Exception =>


[44/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java
index 71567ea,0000000..ff6ecd2
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/IncludeFilterExecuterImpl.java
@@@ -1,224 -1,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) {
++              filterValues[i], false);
++      if (start < 0) {
 +        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) {
++            filterValues[k], false);
++        if (start < 0) {
 +          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/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
index 0463978,0000000..9199907
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
@@@ -1,331 -1,0 +1,379 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + */
 +package org.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.block.SegmentProperties;
 +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.KeyGenException;
 +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.executor.infos.KeyStructureInfo;
++import org.carbondata.scan.executor.util.QueryUtil;
 +import org.carbondata.scan.expression.Expression;
++import org.carbondata.scan.expression.exception.FilterIllegalMemberException;
 +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;
++  protected SegmentProperties segmentProperties;
++  /**
++   * it has index at which given dimension is stored in file
++   */
++  private int[] blocksIndex;
 +
 +  public RowLevelFilterExecuterImpl(List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
 +      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-       AbsoluteTableIdentifier tableIdentifier) {
++      AbsoluteTableIdentifier tableIdentifier, SegmentProperties segmentProperties) {
 +    this.dimColEvaluatorInfoList = dimColEvaluatorInfoList;
++    this.segmentProperties = segmentProperties;
++    this.blocksIndex = new int[dimColEvaluatorInfoList.size()];
++    for (int i=0;i<dimColEvaluatorInfoList.size();i++) {
++      this.blocksIndex[i] = segmentProperties.getDimensionOrdinalToBlockMapping()
++              .get(dimColEvaluatorInfoList.get(i).getColumnIndex());
++    }
 +    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) {
++    for (int i = 0; i < dimColEvaluatorInfoList.size(); i++) {
++      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
 +      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());
++        if (null == blockChunkHolder.getDimensionDataChunk()[blocksIndex[i]]) {
++          blockChunkHolder.getDimensionDataChunk()[blocksIndex[i]] = blockChunkHolder.getDataBlock()
++              .getDimensionChunk(blockChunkHolder.getFileReader(), blocksIndex[i]);
 +        }
 +      } else {
 +        GenericQueryType complexType = dimColumnEvaluatorInfo.getComplexTypesWithBlockStartIndex()
-             .get(dimColumnEvaluatorInfo.getColumnIndex());
++            .get(blocksIndex[i]);
 +        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
++    boolean invalidRowsPresent = false;
 +    for (int index = 0; index < numberOfRows; index++) {
 +      try {
 +        createRow(blockChunkHolder, row, index);
-       } catch (QueryExecutionException e1) {
-         // TODO Auto-generated catch block
-         e1.printStackTrace();
++      } catch (QueryExecutionException e) {
++        FilterUtil.logError(e, invalidRowsPresent);
 +      }
++      Boolean rslt = false;
 +      try {
-         Boolean rslt = exp.evaluate(row).getBoolean();
-         if (null != rslt && rslt) {
-           set.set(index);
-         }
-       } catch (FilterUnsupportedException e) {
-         throw new FilterUnsupportedException(e.getMessage());
++        rslt = exp.evaluate(row).getBoolean();
++      }
++      // Any invalid member while evaluation shall be ignored, system will log the
++      // error only once since all rows the evaluation happens so inorder to avoid
++      // too much log inforation only once the log will be printed.
++      catch (FilterIllegalMemberException e) {
++        FilterUtil.logError(e, invalidRowsPresent);
++      }
++      if (null != rslt && rslt) {
++        set.set(index);
 +      }
 +    }
-     // 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) {
++    for (int i=0;i<dimColEvaluatorInfoList.size();i++) {
++      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo = dimColEvaluatorInfoList.get(i);
 +      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) {
++            && blockChunkHolder
++            .getDimensionDataChunk()[blocksIndex[i]] instanceof VariableLengthDimensionDataChunk) {
 +
 +          VariableLengthDimensionDataChunk dimensionColumnDataChunk =
 +              (VariableLengthDimensionDataChunk) blockChunkHolder
-                   .getDimensionDataChunk()[dimColumnEvaluatorInfo.getColumnIndex()];
++                  .getDimensionDataChunk()[blocksIndex[i]];
 +          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);
++              readSurrogatesFromColumnBlock(blockChunkHolder, index, dimColumnEvaluatorInfo,
++                  blocksIndex[i]);
 +          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;
++        record[msrColumnEvalutorInfo.getRowIndex()] =
++            blockChunkHolder.getMeasureDataChunk()[msrColumnEvalutorInfo.getColumnIndex()]
++                .getNullValueIndexHolder().getBitSet().get(index) ? null : 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;
++      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int blockIndex) {
++    if (dimColumnEvaluatorInfo.getDimension().isColumnar()) {
++      byte[] rawData = blockChunkHolder.getDimensionDataChunk()[blockIndex].getChunkData(index);
++      ByteBuffer byteBuffer = ByteBuffer.allocate(CarbonCommonConstants.INT_SIZE_IN_BYTE);
++      int dictionaryValue = CarbonUtil.getSurrogateKey(rawData, byteBuffer);
++      return dictionaryValue;
++    } else {
++      return readSurrogatesFromColumnGroupBlock(blockChunkHolder, index, dimColumnEvaluatorInfo,
++          blockIndex);
++    }
++
++  }
++
++  /**
++   * @param blockChunkHolder
++   * @param index
++   * @param dimColumnEvaluatorInfo
++   * @return read surrogate of given row of given column group dimension
++   */
++  private int readSurrogatesFromColumnGroupBlock(BlocksChunkHolder blockChunkHolder, int index,
++      DimColumnResolvedFilterInfo dimColumnEvaluatorInfo, int blockIndex) {
++    try {
++      KeyStructureInfo keyStructureInfo =
++          QueryUtil.getKeyStructureInfo(segmentProperties, dimColumnEvaluatorInfo);
++      byte[] colData = blockChunkHolder.getDimensionDataChunk()[blockIndex].getChunkData(index);
++      long[] result = keyStructureInfo.getKeyGenerator().getKeyArray(colData);
++      int colGroupId =
++          QueryUtil.getColumnGroupId(segmentProperties, dimColumnEvaluatorInfo.getColumnIndex());
++      int dictionaryValue = (int) result[segmentProperties
++          .getColumnGroupMdKeyOrdinal(colGroupId, dimColumnEvaluatorInfo.getColumnIndex())];
++      return dictionaryValue;
++    } catch (KeyGenException e) {
++      LOGGER.error(e);
++    }
++    return 0;
 +  }
 +
 +  /**
 +   * 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/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
index 2681ccc,0000000..0f2eda1
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@@ -1,66 -1,0 +1,208 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.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.metadata.encoder.Encoding;
 +import org.carbondata.core.util.ByteUtil;
++import org.carbondata.core.util.CarbonUtil;
 +import org.carbondata.scan.expression.Expression;
++import org.carbondata.scan.expression.exception.FilterUnsupportedException;
 +import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 +import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
++import org.carbondata.scan.processor.BlocksChunkHolder;
 +
 +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);
++      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
++      SegmentProperties segmentProperties) {
++    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
 +    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;
 +
 +  }
++
++  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
++      throws FilterUnsupportedException {
++    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
++      return super.applyFilter(blockChunkHolder);
++    }
++    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
++        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
++    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
++      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
++          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
++    }
++    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
++        blockChunkHolder.getDataBlock().nodeSize());
++  }
++
++  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows) {
++    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
++        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++      return setFilterdIndexToBitSetWithColumnIndex(
++          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
++    }
++    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all members
++   * will be considered for applying range filters. this method will be called if the
++   * column is not supported by default so column index mapping  will be present for
++   * accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @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 = this.filterRangeValues;
++    for (int i = 0; i < filterValues.length; i++) {
++      start = CarbonUtil
++          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++              filterValues[i], true);
++      if (start >= 0) {
++        start = CarbonUtil.nextGreaterValueToTarget(start,
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[i], numerOfRows);
++      }
++      // Logic will handle the case where the range filter member is not present in block
++      // in this case the binary search will return the index from where the bit sets will be
++      // set inorder to apply filters. this is greater than filter so the range will be taken
++      // from the next element which is greater than filter member.
++      if (start < 0) {
++        start = -(start + 1);
++        if (start == numerOfRows) {
++          start = start - 1;
++        }
++        // Method will compare the tentative index value after binary search, this tentative
++        // index needs to be compared by the filter member if its > filter then from that
++        // index the bitset will be considered for filtering process.
++        if (ByteUtil
++            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
++            > 0) {
++          start = start + 1;
++        }
++      }
++
++      last = start;
++      for (int j = start; j < numerOfRows; j++) {
++        bitSet.set(columnIndex[j]);
++        last++;
++      }
++      startIndex = last;
++      if (startIndex >= numerOfRows) {
++        break;
++      }
++    }
++
++    return bitSet;
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all
++   * members will be considered for applying range filters. this method will
++   * be called if the column is sorted default so column index
++   * mapping will be present for accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @return BitSet.
++   */
++  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows) {
++    BitSet bitSet = new BitSet(numerOfRows);
++    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++      int start = 0;
++      int last = 0;
++      int startIndex = 0;
++      byte[][] filterValues = this.filterRangeValues;
++      for (int k = 0; k < filterValues.length; k++) {
++        start = CarbonUtil.getFirstIndexUsingBinarySearch(
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++            filterValues[k], true);
++        start = CarbonUtil.nextGreaterValueToTarget(start,
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[k], numerOfRows);
++        if (start < 0) {
++          start = -(start + 1);
++          if (start == numerOfRows) {
++            start = start - 1;
++          }
++          // Method will compare the tentative index value after binary search, this tentative
++          // index needs to be compared by the filter member if its > filter then from that
++          // index the bitset will be considered for filtering process.
++          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) > 0) {
++            start = start + 1;
++          }
++        }
++        last = start;
++        for (int j = start; j < numerOfRows; j++) {
++          bitSet.set(j);
++          last++;
++        }
++        startIndex = last;
++        if (startIndex >= numerOfRows) {
++          break;
++        }
++      }
++    }
++    return bitSet;
++  }
++
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
index ef11b4a,0000000..e715261
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@@ -1,66 -1,0 +1,199 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.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.metadata.encoder.Encoding;
 +import org.carbondata.core.util.ByteUtil;
++import org.carbondata.core.util.CarbonUtil;
 +import org.carbondata.scan.expression.Expression;
++import org.carbondata.scan.expression.exception.FilterUnsupportedException;
 +import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 +import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
++import org.carbondata.scan.processor.BlocksChunkHolder;
 +
 +public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl {
 +
-   private byte[][] filterRangeValues;
++  protected byte[][] filterRangeValues;
 +
 +  public RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
 +      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
 +      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
-     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
++      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
++      SegmentProperties segmentProperties) {
++    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
 +    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;
 +
 +  }
++
++  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
++      throws FilterUnsupportedException {
++    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
++      return super.applyFilter(blockChunkHolder);
++    }
++    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
++        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
++    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
++      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
++          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
++    }
++    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
++        blockChunkHolder.getDataBlock().nodeSize());
++  }
++
++  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows) {
++    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
++        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++      return setFilterdIndexToBitSetWithColumnIndex(
++          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
++    }
++    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all members
++   * will be considered for applying range filters. this method will be called if the
++   * column is not supported by default so column index mapping  will be present for
++   * accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @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 = this.filterRangeValues;
++    for (int i = 0; i < filterValues.length; i++) {
++      start = CarbonUtil
++          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++              filterValues[i], false);
++      if (start < 0) {
++        start = -(start + 1);
++        if (start == numerOfRows) {
++          start = start - 1;
++        }
++        // Method will compare the tentative index value after binary search, this tentative
++        // index needs to be compared by the filter member if its >= filter then from that
++        // index the bitset will be considered for filtering process.
++        if (ByteUtil
++            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
++            >= 0) {
++          start = start + 1;
++        }
++      }
++      last = start;
++      for (int j = start; j < numerOfRows; j++) {
++
++        bitSet.set(columnIndex[j]);
++        last++;
++      }
++      startIndex = last;
++      if (startIndex >= numerOfRows) {
++        break;
++      }
++    }
++    return bitSet;
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all
++   * members will be considered for applying range filters. this method will
++   * be called if the column is sorted default so column index
++   * mapping will be present for accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @return BitSet.
++   */
++  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows) {
++    BitSet bitSet = new BitSet(numerOfRows);
++    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++      int start = 0;
++      int last = 0;
++      int startIndex = 0;
++      byte[][] filterValues = this.filterRangeValues;
++      for (int k = 0; k < filterValues.length; k++) {
++        start = CarbonUtil.getFirstIndexUsingBinarySearch(
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++            filterValues[k], false);
++        if (start < 0) {
++          start = -(start + 1);
++          if (start == numerOfRows) {
++            start = start - 1;
++          }
++          // Method will compare the tentative index value after binary search, this tentative
++          // index needs to be compared by the filter member if its >= filter then from that
++          // index the bitset will be considered for filtering process.
++          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start))
++              >= 0) {
++            start = start + 1;
++          }
++        }
++
++        last = start;
++        for (int j = start; j < numerOfRows; j++) {
++          bitSet.set(j);
++          last++;
++        }
++        startIndex = last;
++        if (startIndex >= numerOfRows) {
++          break;
++        }
++      }
++    }
++    return bitSet;
++  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
index afc1ccb,0000000..92efb0a
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@@ -1,66 -1,0 +1,247 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.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.metadata.encoder.Encoding;
++import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
++import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 +import org.carbondata.core.util.ByteUtil;
++import org.carbondata.core.util.CarbonUtil;
 +import org.carbondata.scan.expression.Expression;
++import org.carbondata.scan.expression.exception.FilterUnsupportedException;
++import org.carbondata.scan.filter.FilterUtil;
 +import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 +import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
++import org.carbondata.scan.processor.BlocksChunkHolder;
 +
 +public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilterExecuterImpl {
-   private byte[][] filterRangeValues;
++  protected byte[][] filterRangeValues;
 +
 +  public RowLevelRangeLessThanEqualFilterExecuterImpl(
 +      List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
 +      List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-       AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
-     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
++      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
++      SegmentProperties segmentProperties) {
++    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
 +    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;
 +
 +  }
 +
++  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
++      throws FilterUnsupportedException {
++    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
++      return super.applyFilter(blockChunkHolder);
++    }
++    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
++        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
++    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
++      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
++          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
++    }
++    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
++        blockChunkHolder.getDataBlock().nodeSize());
++  }
++
++  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows) {
++    byte[] defaultValue = null;
++    if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
++      DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
++          .getDirectDictionaryGenerator(
++              dimColEvaluatorInfoList.get(0).getDimension().getDataType());
++      int key = directDictionaryGenerator.generateDirectSurrogateKey(null) + 1;
++      defaultValue = FilterUtil.getMaskKey(key, dimColEvaluatorInfoList.get(0).getDimension(),
++          this.segmentProperties.getDimensionKeyGenerator());
++    }
++    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
++        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++
++      return setFilterdIndexToBitSetWithColumnIndex(
++          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows, defaultValue);
++
++    }
++    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all members
++   * will be considered for applying range filters. this method will be called if the
++   * column is not supported by default so column index mapping  will be present for
++   * accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @return BitSet.
++   */
++  private BitSet setFilterdIndexToBitSetWithColumnIndex(
++      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows,
++      byte[] defaultValue) {
++    BitSet bitSet = new BitSet(numerOfRows);
++    int[] columnIndex = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
++    int start = 0;
++    int last = 0;
++    int skip = 0;
++    int startIndex = 0;
++    byte[][] filterValues = this.filterRangeValues;
++    //find the number of default values to skip the null value in case of direct dictionary
++    if (null != defaultValue) {
++      start = CarbonUtil
++          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++              defaultValue, true);
++      if (start < 0) {
++        skip = -(start + 1);
++        // end of block
++        if (skip == numerOfRows) {
++          return bitSet;
++        }
++      } else {
++        skip = start;
++      }
++      startIndex = skip;
++    }
++    for (int i = 0; i < filterValues.length; i++) {
++      start = CarbonUtil
++          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++              filterValues[i], true);
++      if (start < 0) {
++        start = -(start + 1);
++        if (start == numerOfRows) {
++          start = start - 1;
++        }
++        // Method will compare the tentative index value after binary search, this tentative
++        // index needs to be compared by the filter member if its >= filter then from that
++        // index the bitset will be considered for filtering process.
++        if (ByteUtil
++            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
++            <= 0) {
++          start = start - 1;
++        }
++      }
++      last = start;
++      for (int j = start; j >= skip; j--) {
++        bitSet.set(columnIndex[j]);
++        last--;
++      }
++      startIndex = last;
++      if (startIndex <= 0) {
++        break;
++      }
++    }
++    return bitSet;
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all
++   * members will be considered for applying range filters. this method will
++   * be called if the column is sorted default so column index
++   * mapping will be present for accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @param defaultValue
++   * @return BitSet.
++   */
++  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows, byte[] defaultValue) {
++    BitSet bitSet = new BitSet(numerOfRows);
++    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++      int start = 0;
++      int last = 0;
++      int startIndex = 0;
++      byte[][] filterValues = this.filterRangeValues;
++      int skip = 0;
++      //find the number of default values to skip the null value in case of direct dictionary
++      if (null != defaultValue) {
++        start = CarbonUtil.getFirstIndexUsingBinarySearch(
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++            defaultValue, true);
++        if (start < 0) {
++          skip = -(start + 1);
++          // end of block
++          if (skip == numerOfRows) {
++            return bitSet;
++          }
++        } else {
++          skip = start;
++        }
++        startIndex = skip;
++      }
++      for (int k = 0; k < filterValues.length; k++) {
++        start = CarbonUtil.getFirstIndexUsingBinarySearch(
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++            filterValues[k], true);
++        if (start < 0) {
++          start = -(start + 1);
++          if (start == numerOfRows) {
++            start = start - 1;
++          }
++          // Method will compare the tentative index value after binary search, this tentative
++          // index needs to be compared by the filter member if its <= filter then from that
++          // index the bitset will be considered for filtering process.
++          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start))
++              <= 0) {
++            start = start - 1;
++          }
++        }
++        last = start;
++        for (int j = start; j >= skip; j--) {
++          bitSet.set(j);
++          last--;
++        }
++        startIndex = last;
++        if (startIndex <= 0) {
++          break;
++        }
++      }
++    }
++    return bitSet;
++  }
++
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
index d608cc6,0000000..46e7d1b
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@@ -1,65 -1,0 +1,251 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.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.metadata.encoder.Encoding;
++import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryGenerator;
++import org.carbondata.core.keygenerator.directdictionary.DirectDictionaryKeyGeneratorFactory;
 +import org.carbondata.core.util.ByteUtil;
++import org.carbondata.core.util.CarbonUtil;
 +import org.carbondata.scan.expression.Expression;
++import org.carbondata.scan.expression.exception.FilterUnsupportedException;
++import org.carbondata.scan.filter.FilterUtil;
 +import org.carbondata.scan.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 +import org.carbondata.scan.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
++import org.carbondata.scan.processor.BlocksChunkHolder;
 +
 +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);
++      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
++      SegmentProperties segmentProperties) {
++    super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier, segmentProperties);
 +    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;
 +
 +  }
++
++  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
++      throws FilterUnsupportedException {
++    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
++      return super.applyFilter(blockChunkHolder);
++    }
++    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
++        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
++    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
++      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
++          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
++    }
++    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
++        blockChunkHolder.getDataBlock().nodeSize());
++  }
++
++  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows) {
++    byte[] defaultValue = null;
++    if (dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
++      DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
++          .getDirectDictionaryGenerator(
++              dimColEvaluatorInfoList.get(0).getDimension().getDataType());
++      int key = directDictionaryGenerator.generateDirectSurrogateKey(null) + 1;
++      defaultValue = FilterUtil.getMaskKey(key, dimColEvaluatorInfoList.get(0).getDimension(),
++          this.segmentProperties.getDimensionKeyGenerator());
++    }
++    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
++        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++      return setFilterdIndexToBitSetWithColumnIndex(
++          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows, defaultValue);
++    }
++    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows, defaultValue);
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all members
++   * will be considered for applying range filters. this method will be called if the
++   * column is not supported by default so column index mapping  will be present for
++   * accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @return BitSet.
++   */
++  private BitSet setFilterdIndexToBitSetWithColumnIndex(
++      FixedLengthDimensionDataChunk dimensionColumnDataChunk, int numerOfRows,
++      byte[] defaultValue) {
++    BitSet bitSet = new BitSet(numerOfRows);
++    int[] columnIndex = dimensionColumnDataChunk.getAttributes().getInvertedIndexes();
++    int start = 0;
++    int last = 0;
++    int startIndex = 0;
++    int skip = 0;
++    byte[][] filterValues = this.filterRangeValues;
++
++    //find the number of default values to skip the null value in case of direct dictionary
++    if (null != defaultValue) {
++      start = CarbonUtil
++          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++              defaultValue, false);
++      if (start < 0) {
++        skip = -(start + 1);
++        // end of block
++        if (skip == numerOfRows) {
++          return bitSet;
++        }
++      } else {
++        skip = start;
++      }
++      startIndex = skip;
++    }
++
++    for (int i = 0; i < filterValues.length; i++) {
++      start = CarbonUtil
++          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++              filterValues[i], false);
++      // Logic will handle the case where the range filter member is not present in block
++      // in this case the binary search will return the index from where the bit sets will be
++      // set inorder to apply filters. this is Lesser than filter so the range will be taken
++      // from the prev element which is Lesser than filter member.
++      start = CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[i]);
++      if (start < 0) {
++        start = -(start + 1);
++        if (start == numerOfRows) {
++          start = start - 1;
++        }
++        // Method will compare the tentative index value after binary search, this tentative
++        // index needs to be compared by the filter member if its < filter then from that
++        // index the bitset will be considered for filtering process.
++        if (ByteUtil
++            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
++            < 0) {
++          start = start - 1;
++        }
++      }
++      last = start;
++      for (int j = start; j >= skip; j--) {
++        bitSet.set(columnIndex[j]);
++        last--;
++      }
++      startIndex = last;
++      if (startIndex >= 0) {
++        break;
++      }
++    }
++    return bitSet;
++  }
++
++  /**
++   * Method will scan the block and finds the range start index from which all
++   * members will be considered for applying range filters. this method will
++   * be called if the column is sorted default so column index
++   * mapping will be present for accesing the members from the block.
++   *
++   * @param dimensionColumnDataChunk
++   * @param numerOfRows
++   * @return BitSet.
++   */
++  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
++      int numerOfRows, byte[] defaultValue) {
++    BitSet bitSet = new BitSet(numerOfRows);
++    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
++      int start = 0;
++      int last = 0;
++      int startIndex = 0;
++      int skip = 0;
++      byte[][] filterValues = this.filterRangeValues;
++      //find the number of default values to skip the null value in case of direct dictionary
++      if (null != defaultValue) {
++        start = CarbonUtil.getFirstIndexUsingBinarySearch(
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++            defaultValue, false);
++        if (start < 0) {
++          skip = -(start + 1);
++          // end of block
++          if (skip == numerOfRows) {
++            return bitSet;
++          }
++        } else {
++          skip = start;
++        }
++        startIndex = skip;
++      }
++      for (int k = 0; k < filterValues.length; k++) {
++        start = CarbonUtil.getFirstIndexUsingBinarySearch(
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
++            filterValues[k], false);
++        start = CarbonUtil.nextLesserValueToTarget(start,
++            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[k]);
++        if (start < 0) {
++          start = -(start + 1);
++          if (start >= numerOfRows) {
++            start = numerOfRows - 1;
++          }
++          // Method will compare the tentative index value after binary search, this tentative
++          // index needs to be compared by the filter member if its < filter then from that
++          // index the bitset will be considered for filtering process.
++          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) {
++            start = start - 1;
++          }
++        }
++        last = start;
++        for (int j = start; j >= skip; j--) {
++          bitSet.set(j);
++          last--;
++        }
++        startIndex = last;
++        if (startIndex <= 0) {
++          break;
++        }
++      }
++    }
++    return bitSet;
++  }
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
index db2fa2d,0000000..e6eb6da
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
+++ b/core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
@@@ -1,90 -1,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.filter.executer;
 +
++import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 +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 segmentProperties
 +   * @param dataType                     DataType
 +   * @return the generator instance
 +   */
 +  public static RowLevelFilterExecuterImpl getRowLevelRangeTypeExecuter(
-       FilterExecuterType filterExecuterType, FilterResolverIntf filterExpressionResolverTree) {
++      FilterExecuterType filterExecuterType, FilterResolverIntf filterExpressionResolverTree,
++      SegmentProperties segmentProperties) {
 +    switch (filterExecuterType) {
 +
 +      case ROWLEVEL_LESSTHAN:
 +        return new RowLevelRangeLessThanFiterExecuterImpl(
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getDimColEvaluatorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getMsrColEvalutorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                 .getFilterRangeValues());
++                .getFilterRangeValues(segmentProperties), segmentProperties);
 +      case ROWLEVEL_LESSTHAN_EQUALTO:
 +        return new RowLevelRangeLessThanEqualFilterExecuterImpl(
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getDimColEvaluatorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getMsrColEvalutorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                 .getFilterRangeValues());
++                .getFilterRangeValues(segmentProperties), segmentProperties);
 +      case ROWLEVEL_GREATERTHAN_EQUALTO:
 +        return new RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getDimColEvaluatorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getMsrColEvalutorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                 .getFilterRangeValues());
++                .getFilterRangeValues(segmentProperties), segmentProperties);
 +      case ROWLEVEL_GREATERTHAN:
 +        return new RowLevelRangeGrtThanFiterExecuterImpl(
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getDimColEvaluatorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
 +                .getMsrColEvalutorInfoList(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
 +            ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                 .getFilterRangeValues());
++                .getFilterRangeValues(segmentProperties), segmentProperties);
 +      default:
 +        // Scenario wont come logic must break
 +        return null;
 +
 +    }
 +  }
 +
 +}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
index 37685c8,0000000..cde5c43
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
@@@ -1,51 -1,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.resolver;
 +
 +import java.util.SortedMap;
 +
 +import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 +import org.carbondata.core.carbon.datastore.block.SegmentProperties;
++import org.carbondata.scan.executor.exception.QueryExecutionException;
 +import org.carbondata.scan.filter.intf.ExpressionType;
 +
 +public class AndFilterResolverImpl extends LogicalFilterResolverImpl {
 +
 +  /**
-    *
++   *i
 +   */
 +  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) {
++      SortedMap<Integer, byte[]> noDicEndKeys) throws QueryExecutionException {
 +    leftEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
 +    rightEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
 +  }
 +}


[40/50] [abbrv] incubator-carbondata git commit: Update README.md

Posted by ch...@apache.org.
Update README.md

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

Branch: refs/heads/master
Commit: 6e3e4c35ab8e2c338c6f3de4c1338083cd9aaa99
Parents: 167d527
Author: Liang Chen <ch...@apache.org>
Authored: Thu Jun 30 06:52:25 2016 +0530
Committer: GitHub <no...@github.com>
Committed: Thu Jun 30 06:52:25 2016 +0530

----------------------------------------------------------------------
 README.md | 9 +--------
 1 file changed, 1 insertion(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6e3e4c35/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index a2cd048..0f7e1d7 100644
--- a/README.md
+++ b/README.md
@@ -1,17 +1,10 @@
+# This github has migrated to apache: https://github.com/apache/incubator-carbondata, please fork new github.
 # CarbonData
 CarbonData is a new Apache Hadoop native file format for faster 
 interactive query using advanced columnar storage, index, compression 
 and encoding techniques to improve computing efficiency, in turn it will 
 help speedup queries an order of magnitude faster over PetaBytes of data. 
 
-### Why CarbonData
-Based on the below requirements, we investigated existing file formats in the Hadoop eco-system, but we could not find a suitable solution that can satisfy all the requirements at the same time,so we start designing CarbonData. 
-* Requirement1:Support big scan & only fetch a few columns 
-* Requirement2:Support primary key lookup response in sub-second. 
-* Requirement3:Support interactive OLAP-style query over big data which involve many filters in a query, this type of workload should response in seconds. 
-* Requirement4:Support fast individual record extraction which fetch all columns of the record. 
-* Requirement5:Support HDFS so that customer can leverage existing Hadoop cluster. 
-
 ### Features
 CarbonData file format is a columnar store in HDFS, it has many features that a modern columnar format has, such as splittable, compression schema ,complex data type etc. And CarbonData has following unique features:
 * Stores data along with index: it can significantly accelerate query performance and reduces the I/O scans and CPU resources, where there are filters in the query.  CarbonData index consists of multiple level of indices, a processing framework can leverage this index to reduce the task it needs to schedule and process, and it can also do skip scan in more finer grain unit (called blocklet) in task side scanning instead of scanning the whole file. 


[32/50] [abbrv] incubator-carbondata git commit: [Bug] Handling failing scenarios & providing proper logs for delete queries (#769)

Posted by ch...@apache.org.
[Bug] Handling failing scenarios & providing proper logs for delete queries (#769)



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

Branch: refs/heads/master
Commit: 6701eeefcdf7f8f16d4cea1ba01588c63780c5b9
Parents: 3718dc2
Author: Manu <ma...@gmail.com>
Authored: Tue Jun 28 04:59:20 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Tue Jun 28 04:59:20 2016 +0530

----------------------------------------------------------------------
 .../execution/command/carbonTableSchema.scala   | 32 +++++-----
 .../lcm/status/SegmentStatusManager.java        | 65 +++++++++-----------
 2 files changed, 43 insertions(+), 54 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6701eeef/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 c97a2fe..ffc7e60 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
@@ -1168,6 +1168,7 @@ private[sql] case class AlterTable(
 
 /**
  * Command for the compaction in alter table command
+ *
  * @param alterTableModel
  */
 private[sql] case class AlterTableCompaction(alterTableModel: AlterTableModel) extends
@@ -1337,25 +1338,15 @@ private[sql] case class DeleteLoadsById(
 
     val invalidLoadIds = segmentStatusManager.updateDeletionStatus(loadids.asJava, path).asScala
 
-    if (invalidLoadIds.nonEmpty) {
-      if (invalidLoadIds.length == loadids.length) {
-        LOGGER.audit(
-          "Delete load by Id is failed. Failed to delete the following load(s). LoadSeqId-" +
-          invalidLoadIds)
-        sys.error("Load deletion is failed. Failed to delete the following load(s). LoadSeqId-" +
-                  invalidLoadIds)
-      }
-      else {
-        LOGGER.audit(
-          "Delete load by Id is failed. Failed to delete the following load(s). LoadSeqId-" +
-          invalidLoadIds)
-        sys.error(
-          "Load deletion is partial success. Failed to delete the following load(s). LoadSeqId-" +
-          invalidLoadIds)
-      }
+    if (invalidLoadIds.isEmpty) {
+
+      LOGGER.audit("Delete load by Id is successfull.")
+    }
+    else {
+      sys.error("Delete load by Id is failed. No matching load id found. SegmentSeqId(s) - "
+                + invalidLoadIds)
     }
 
-    LOGGER.audit("Delete load by Id is successfull.")
     Seq.empty
 
   }
@@ -1416,7 +1407,12 @@ private[sql] case class DeleteLoadsByLoadDate(
 
     var invalidLoadTimestamps = segmentStatusManager
       .updateDeletionStatus(loadDate, path, timeObj.asInstanceOf[java.lang.Long]).asScala
-    LOGGER.audit("Delete load by load date is successfull.")
+    if(invalidLoadTimestamps.isEmpty) {
+      LOGGER.audit("Delete load by load date is successfull.")
+    }
+    else {
+      sys.error("Delete load by load date is failed. No matching load found.")
+    }
     Seq.empty
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/6701eeef/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 8708779..48964ca 100644
--- a/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
@@ -253,12 +253,18 @@ public class SegmentStatusManager {
         listOfLoadFolderDetailsArray = readLoadMetadata(cubeFolderPath);
         if (listOfLoadFolderDetailsArray != null && listOfLoadFolderDetailsArray.length != 0) {
           updateDeletionStatus(loadIds, listOfLoadFolderDetailsArray, invalidLoadIds);
-          if (!invalidLoadIds.isEmpty()) {
-            LOG.warn("Load doesnt exist or it is already deleted , LoadSeqId-" + invalidLoadIds);
+          if(invalidLoadIds.isEmpty())
+          {
+            // All or None , if anything fails then dont write
+            writeLoadDetailsIntoFile(dataLoadLocation, listOfLoadFolderDetailsArray);
           }
-          writeLoadDetailsIntoFile(dataLoadLocation, listOfLoadFolderDetailsArray);
+          else
+          {
+            return invalidLoadIds;
+          }
+
         } else {
-          LOG.warn("Load doesnt exist or it is already deleted , LoadSeqId-" + loadIds);
+          LOG.audit("Delete load by Id is failed. No matching load id found.");
           return loadIds;
         }
 
@@ -307,24 +313,16 @@ public class SegmentStatusManager {
         if (listOfLoadFolderDetailsArray != null && listOfLoadFolderDetailsArray.length != 0) {
           updateDeletionStatus(loadDate, listOfLoadFolderDetailsArray,
               invalidLoadTimestamps, loadStartTime);
-          if (!invalidLoadTimestamps.isEmpty()) {
-            LOG.warn("Load doesnt exist or it is already deleted , LoadTimestamps-"
-                + invalidLoadTimestamps);
-            if (invalidLoadTimestamps.size() == listOfLoadFolderDetailsArray.length) {
-              LOG.audit(
-                  "The delete load by Id is failed. Failed to delete the following load(s)."
-                      + " LoadSeqId-" + invalidLoadTimestamps);
-              LOG.error("Error message: "
-                  + "Load deletion is failed. Failed to delete the following load(s). LoadSeqId-" +
-                  invalidLoadTimestamps);
-
-            }
+          if(invalidLoadTimestamps.isEmpty()) {
+            writeLoadDetailsIntoFile(dataLoadLocation, listOfLoadFolderDetailsArray);
+          }
+          else
+          {
+            return invalidLoadTimestamps;
           }
-
-          writeLoadDetailsIntoFile(dataLoadLocation, listOfLoadFolderDetailsArray);
 
         } else {
-          LOG.warn("Load doesnt exist or it is already deleted , LoadTimestamp-" + loadDate);
+          LOG.audit("Delete load by date is failed. No matching load found.");
           invalidLoadTimestamps.add(loadDate);
           return invalidLoadTimestamps;
         }
@@ -383,7 +381,7 @@ public class SegmentStatusManager {
    * @param invalidLoadIds
    * @return invalidLoadIds
    */
-  public void updateDeletionStatus(List<String> loadIds,
+  public List<String> updateDeletionStatus(List<String> loadIds,
       LoadMetadataDetails[] listOfLoadFolderDetailsArray, List<String> invalidLoadIds) {
     for (String loadId : loadIds) {
       boolean loadFound = false;
@@ -393,26 +391,24 @@ public class SegmentStatusManager {
       for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
 
         if (loadId.equalsIgnoreCase(loadMetadata.getLoadName())) {
-          loadFound = true;
           if (!CarbonCommonConstants.MARKED_FOR_DELETE.equals(loadMetadata.getLoadStatus())) {
+            loadFound = true;
             loadMetadata.setLoadStatus(CarbonCommonConstants.MARKED_FOR_DELETE);
             loadMetadata.setModificationOrdeletionTimesStamp(readCurrentTime());
             LOG.info("LoadId " + loadId + " Marked for Delete");
-          } else {
-            // it is already deleted . can not delete it again.
-            invalidLoadIds.add(loadId);
           }
-
           break;
         }
       }
 
       if (!loadFound) {
+        LOG.audit("Delete load by Id is failed. No matching load id found.");
         invalidLoadIds.add(loadId);
+        return invalidLoadIds;
       }
 
     }
-
+    return invalidLoadIds;
   }
 
   /**
@@ -423,7 +419,7 @@ public class SegmentStatusManager {
    * @param invalidLoadTimestamps
    * @return invalidLoadTimestamps
    */
-  public void updateDeletionStatus(String loadDate,
+  public List<String> updateDeletionStatus(String loadDate,
       LoadMetadataDetails[] listOfLoadFolderDetailsArray, List<String> invalidLoadTimestamps,
       Long loadStartTime) {
     // For each load timestamp loop through data and if the
@@ -433,28 +429,25 @@ public class SegmentStatusManager {
     String loadStartTimeString = "Load Start Time: ";
     for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
       Integer result = compareDateValues(loadMetadata.getLoadStartTimeAsLong(), loadStartTime);
-      if (null == result) {
-        invalidLoadTimestamps.add(loadDate);
-      } else if (result < 0) {
-        loadFound = true;
+      if (result < 0) {
         if (!CarbonCommonConstants.MARKED_FOR_DELETE.equals(loadMetadata.getLoadStatus())) {
+          loadFound = true;
           loadMetadata.setLoadStatus(CarbonCommonConstants.MARKED_FOR_DELETE);
           loadMetadata.setModificationOrdeletionTimesStamp(readCurrentTime());
           LOG.info("Info: " +
               loadStartTimeString + loadMetadata.getLoadStartTime() +
               " Marked for Delete");
-        } else {
-          // it is already deleted . can not delete it again.
-          invalidLoadTimestamps.add(loadMetadata.getLoadStartTime());
         }
       }
+
     }
 
     if (!loadFound) {
       invalidLoadTimestamps.add(loadDate);
-
+      LOG.audit("Delete load by date is failed. No matching load found.");
+      return invalidLoadTimestamps;
     }
-
+    return invalidLoadTimestamps;
   }
 
   /**


[50/50] [abbrv] incubator-carbondata git commit: Code merge from HuaweiBigData/master to apache/master This closes #6

Posted by ch...@apache.org.
Code merge from HuaweiBigData/master to apache/master This closes #6


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

Branch: refs/heads/master
Commit: 6e943ff73a293a84aeea8fc69802703499d81800
Parents: e9bbf75 4444c32
Author: chenliang613 <ch...@apache.org>
Authored: Thu Jun 30 23:11:12 2016 +0530
Committer: chenliang613 <ch...@apache.org>
Committed: Thu Jun 30 23:11:12 2016 +0530

----------------------------------------------------------------------
 common/pom.xml                                  |    8 +-
 .../common/logging/impl/StandardLogService.java |   15 +-
 conf/carbon.properties.template                 |   98 +
 .../common/ext/ColumnUniqueIdGenerator.java     |   41 +
 .../common/ext/DictionaryFactory.java           |  119 +
 .../org/carbondata/common/ext/PathFactory.java  |   48 +
 .../common/factory/CarbonCommonFactory.java     |   54 +
 .../dictionary/AbstractDictionaryCache.java     |   43 +-
 .../cache/dictionary/ColumnDictionaryInfo.java  |    5 +-
 .../cache/dictionary/DictionaryCacheLoader.java |    7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |   28 +-
 .../DictionaryColumnUniqueIdentifier.java       |    9 +-
 .../dictionary/ForwardDictionaryCache.java      |   11 +-
 .../dictionary/ReverseDictionaryCache.java      |   11 +-
 .../core/carbon/ColumnIdentifier.java           |  109 +
 .../core/carbon/datastore/BlockIndexStore.java  |  184 +-
 .../carbon/datastore/SegmentTaskIndexStore.java |   21 +-
 .../datastore/block/SegmentProperties.java      |   94 +-
 .../carbon/datastore/block/TableBlockInfo.java  |   18 +-
 .../impl/ColumnGroupDimensionDataChunk.java     |    7 +-
 .../ThriftWrapperSchemaConverterImpl.java       |    4 +
 .../carbon/metadata/index/BlockIndexInfo.java   |   92 +
 .../metadata/schema/table/CarbonTable.java      |    2 +-
 .../schema/table/column/CarbonColumn.java       |   23 +
 .../schema/table/column/ColumnSchema.java       |   49 +
 .../core/carbon/path/CarbonTablePath.java       |  120 +-
 .../core/constants/CarbonCommonConstants.java   |  285 +-
 .../store/columnar/ColumnGroupModel.java        |   21 -
 .../store/filesystem/HDFSCarbonFile.java        |    3 +-
 .../store/filesystem/ViewFSCarbonFile.java      |    6 +-
 .../core/datastorage/util/StoreFactory.java     |   18 +-
 .../DirectDictionaryGenerator.java              |   13 +
 .../TimeStampDirectDictionaryGenerator.java     |   46 +-
 .../core/load/LoadMetadataDetails.java          |   43 +
 .../CarbonDictionaryMetadataReaderImpl.java     |   17 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |   28 +-
 .../core/reader/CarbonIndexFileReader.java      |   95 +
 .../carbondata/core/reader/ThriftReader.java    |   47 +-
 .../CarbonDictionarySortIndexReaderImpl.java    |   21 +-
 .../core/service/ColumnUniqueIdService.java     |   34 +
 .../core/service/DictionaryService.java         |   92 +
 .../carbondata/core/service/PathService.java    |   38 +
 .../core/util/CarbonMetadataUtil.java           |  114 +-
 .../carbondata/core/util/CarbonProperties.java  |  401 +--
 .../org/carbondata/core/util/CarbonUtil.java    |  245 +-
 .../core/util/DataFileFooterConverter.java      |   65 +-
 .../core/writer/CarbonDictionaryWriterImpl.java |   45 +-
 .../core/writer/CarbonIndexFileWriter.java      |   64 +
 .../core/writer/HierarchyValueWriterForCSV.java |    2 +-
 .../CarbonDictionarySortIndexWriterImpl.java    |   21 +-
 .../scan/executor/infos/KeyStructureInfo.java   |   18 -
 .../scan/executor/util/QueryUtil.java           |  105 +-
 .../scan/expression/ColumnExpression.java       |    3 +-
 .../carbondata/scan/expression/Expression.java  |    4 +-
 .../scan/expression/ExpressionResult.java       |  104 +-
 .../expression/arithmetic/AddExpression.java    |    4 +-
 .../expression/arithmetic/DivideExpression.java |    4 +-
 .../arithmetic/MultiplyExpression.java          |    4 +-
 .../arithmetic/SubstractExpression.java         |    4 +-
 .../conditional/EqualToExpression.java          |    5 +-
 .../GreaterThanEqualToExpression.java           |    4 +-
 .../conditional/GreaterThanExpression.java      |    5 +-
 .../expression/conditional/InExpression.java    |    6 +-
 .../conditional/LessThanEqualToExpression.java  |    5 +-
 .../conditional/LessThanExpression.java         |    4 +-
 .../expression/conditional/ListExpression.java  |    7 +-
 .../conditional/NotEqualsExpression.java        |    4 +-
 .../expression/conditional/NotInExpression.java |    6 +-
 .../exception/FilterIllegalMemberException.java |   98 +
 .../exception/FilterUnsupportedException.java   |    1 -
 .../scan/expression/logical/AndExpression.java  |    6 +-
 .../scan/expression/logical/NotExpression.java  |    4 +-
 .../scan/expression/logical/OrExpression.java   |    4 +-
 .../scan/filter/FilterExpressionProcessor.java  |   36 +-
 .../org/carbondata/scan/filter/FilterUtil.java  |  163 +-
 .../executer/ColGroupFilterExecuterImpl.java    |  190 --
 .../ExcludeColGroupFilterExecuterImpl.java      |  137 +
 .../executer/ExcludeFilterExecuterImpl.java     |   49 +-
 .../IncludeColGroupFilterExecuterImpl.java      |  209 ++
 .../executer/IncludeFilterExecuterImpl.java     |    8 +-
 .../executer/RowLevelFilterExecuterImpl.java    |  116 +-
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |  148 +-
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |  139 +-
 ...velRangeLessThanEqualFilterExecuterImpl.java |  187 +-
 .../RowLevelRangeLessThanFiterExecuterImpl.java |  190 +-
 .../RowLevelRangeTypeExecuterFacory.java        |   13 +-
 .../filter/resolver/AndFilterResolverImpl.java  |    5 +-
 .../resolver/ConditionalFilterResolverImpl.java |   19 +-
 .../filter/resolver/FilterResolverIntf.java     |    5 +-
 .../resolver/LogicalFilterResolverImpl.java     |    3 +-
 .../resolver/RestructureFilterResolverImpl.java |    5 +-
 .../RowLevelRangeFilterResolverImpl.java        |   77 +-
 .../visitor/CustomTypeDictionaryVisitor.java    |   16 +-
 .../visitor/DictionaryColumnVisitor.java        |   11 +-
 .../visitor/NoDictionaryTypeVisitor.java        |   12 +-
 .../scan/result/iterator/RawResultIterator.java |    2 +
 .../org/carbondata/scan/util/DataTypeUtil.java  |    2 +-
 .../dictionary/AbstractDictionaryCacheTest.java |    7 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |    4 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |    4 +-
 .../datastore/SegmentTaskIndexStoreTest.java    |  143 -
 ...CarbonDictionarySortIndexReaderImplTest.java |    6 +-
 .../writer/CarbonDictionaryWriterImplTest.java  |    9 +-
 .../core/writer/CarbonFooterWriterTest.java     |   58 +-
 ...CarbonDictionarySortIndexWriterImplTest.java |   13 +-
 dev/molap.properties.template                   |   94 -
 dev/molap.properties_spark                      |   90 -
 .../examples/GenerateDictionaryExample.scala    |    4 +-
 format/src/main/thrift/carbondataindex.thrift   |   45 +
 format/src/main/thrift/schema.thrift            |   13 +-
 .../org/carbondata/hadoop/CarbonPathFilter.java |    4 +-
 .../AbstractDictionaryDecodedReadSupport.java   |    4 +-
 .../hadoop/test/util/StoreCreator.java          |   11 +-
 .../FACT_UNITED_DATA_INFO_sample_cube.csv       |   20 -
 .../FACT_UNITED_DATA_INFO_sample_table.csv      |   20 +
 .../src/test/resources/restructure_cube.csv     |  101 -
 .../src/test/resources/restructure_table.csv    |  101 +
 .../allqueries/AllDataTypesTestCase1.scala      |   82 +-
 .../allqueries/AllDataTypesTestCase2.scala      |  116 +-
 .../allqueries/AllDataTypesTestCase3.scala      |  211 +-
 .../allqueries/AllDataTypesTestCase4.scala      | 2576 +++++-------------
 .../allqueries/AllDataTypesTestCase5.scala      |  165 +-
 .../allqueries/AllDataTypesTestCase6.scala      | 2377 ++++------------
 .../spark/merger/RowResultMerger.java           |   20 +-
 .../carbondata/spark/load/CarbonLoadModel.java  |    2 +
 .../carbondata/spark/load/CarbonLoaderUtil.java |  202 +-
 .../spark/load/DeleteLoadFolders.java           |    5 +-
 .../spark/merger/CarbonDataMergerUtil.java      |  401 +--
 .../spark/sql/CarbonDatasourceRelation.scala    |   22 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |   17 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  376 +--
 .../execution/command/carbonTableSchema.scala   |  179 +-
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |   85 +-
 .../spark/sql/hive/CarbonStrategies.scala       |   45 +-
 .../spark/CarbonColumnValidator.scala           |   36 +
 .../carbondata/spark/CarbonSparkFactory.scala   |   63 +
 .../spark/DictionaryDetailHelper.scala          |   66 +
 .../spark/rdd/CarbonDataLoadRDD.scala           |    4 +
 .../spark/rdd/CarbonDataRDDFactory.scala        |  240 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |    7 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   13 +-
 .../org/carbondata/spark/rdd/Compactor.scala    |   19 +-
 .../org/carbondata/spark/util/CommonUtil.scala  |   79 +-
 .../spark/util/GlobalDictionaryUtil.scala       |   85 +-
 .../spark/load/CarbonLoaderUtilTest.java        |    4 +-
 .../validation/FileFooterValidator.java         |    6 +-
 integration/spark/src/test/resources/data2.csv  |    6 +-
 .../src/test/resources/data2_DiffTimeFormat.csv |    4 +
 .../resources/datawithcomplexspecialchar.csv    |  151 +
 .../test/resources/datawithescapecharacter.csv  |   22 +
 .../src/test/resources/datawithnullmeasure.csv  |    5 +
 .../test/resources/datawithspecialcharacter.csv |   38 +
 .../spark/src/test/resources/filter/emp2.csv    |    9 +
 .../src/test/resources/filter/emp2allnull.csv   |    9 +
 .../src/test/resources/filter/emp2nonull.csv    |    8 +
 .../test/resources/nullandnonparsableValue.csv  |    3 +
 .../apache/spark/sql/TestCarbonSqlParser.scala  |   37 +-
 .../dataload/TestLoadDataWithBlankLine.scala    |   18 +-
 .../TestLoadDataWithEmptyArrayColumns.scala     |   12 +-
 .../TestLoadDataWithMaxMinInteger.scala         |    6 +-
 .../dataload/TestLoadDataWithNullMeasures.scala |   15 +-
 .../AllDataTypesTestCaseAggregate.scala         |   44 +-
 .../aggquery/IntegerDataTypeTestCase.scala      |    7 +-
 .../AllDataTypesTestCaseAggregate.scala         |    8 +-
 .../testsuite/bigdecimal/TestBigDecimal.scala   |   10 +
 .../bigdecimal/TestNullAndEmptyFields.scala     |  110 +
 .../createtable/TestCreateTableSyntax.scala     |   32 +-
 .../DataCompactionCardinalityBoundryTest.scala  |   12 +-
 .../datacompaction/DataCompactionLockTest.scala |  111 +
 .../datacompaction/DataCompactionTest.scala     |   12 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |  108 +-
 ...ataWithMalformedCarbonCommandException.scala |    4 +-
 .../dataretention/DataRetentionTestCase.scala   |   37 +
 .../detailquery/AllDataTypesTestCase.scala      |    6 +-
 .../ColumnGroupDataTypesTestCase.scala          |   63 +-
 .../ColumnPropertyValidationTestCase.scala      |   28 +
 .../HighCardinalityDataTypesTestCase.scala      |   66 +-
 .../detailquery/IntegerDataTypeTestCase.scala   |    6 +-
 ...estampDataTypeDirectDictionaryTestCase.scala |   12 +-
 ...TypeDirectDictionaryWithNoDictTestCase.scala |    4 +-
 .../filterexpr/AllDataTypesTestCaseFilter.scala |    6 +-
 .../filterexpr/FilterProcessorTestCase.scala    |  131 +-
 .../GrtLtFilterProcessorTestCase.scala          |  176 ++
 .../filterexpr/IntegerDataTypeTestCase.scala    |    6 +-
 .../NullMeasureValueTestCaseFilter.scala        |   39 +
 .../HadoopFSRelationTestCase.scala              |   16 +-
 .../joinquery/AllDataTypesTestCaseJoin.scala    |    6 +-
 .../joinquery/IntegerDataTypeTestCase.scala     |    6 +-
 .../NullMeasureValueTestCaseAggregate.scala     |    2 +-
 .../sortexpr/AllDataTypesTestCaseSort.scala     |    6 +-
 .../sortexpr/IntegerDataTypeTestCase.scala      |   13 +-
 .../util/GlobalDictionaryUtilTestCase.scala     |   51 +-
 .../carbonaggregatesurrogategenerator/icon.png  |  Bin 9058 -> 0 bytes
 .../plugin.xml                                  |    1 -
 .../steps/carbonautoagggraphgenerator/icon.png  |  Bin 9058 -> 0 bytes
 .../carbonautoagggraphgenerator/plugin.xml      |    1 -
 .../steps/carbonautoaggslicemerger/icon.png     |  Bin 9058 -> 0 bytes
 .../steps/carbonautoaggslicemerger/plugin.xml   |    1 -
 .../plugins/steps/carboncsvbasedseqgen/icon.png |  Bin 24093 -> 0 bytes
 .../steps/carboncsvbasedseqgen/plugin.xml       |    1 -
 .../plugins/steps/carboncsvreader/icon.png      |  Bin 24093 -> 0 bytes
 .../plugins/steps/carboncsvreader/plugin.xml    |    1 -
 .../plugins/steps/carboncsvreaderstrep/icon.png |  Bin 9058 -> 0 bytes
 .../steps/carboncsvreaderstrep/plugin.xml       |    1 -
 .../steps/carbondatawriter/Binary-icon.png      |  Bin 14573 -> 0 bytes
 .../plugins/steps/carbondatawriter/plugin.xml   |    1 -
 .../plugins/steps/carbonfactreader/icon.png     |  Bin 9058 -> 0 bytes
 .../plugins/steps/carbonfactreader/plugin.xml   |    1 -
 .../plugins/steps/carbongroupby/icon.png        |  Bin 9058 -> 0 bytes
 .../plugins/steps/carbongroupby/plugin.xml      |    1 -
 .../steps/carboninmemoryfactreader/icon.png     |  Bin 9058 -> 0 bytes
 .../steps/carboninmemoryfactreader/plugin.xml   |    1 -
 .../.kettle/plugins/steps/carbonseqgen/icon.png |  Bin 24093 -> 0 bytes
 .../plugins/steps/carbonseqgen/plugin.xml       |    1 -
 .../plugins/steps/carbonslicemerger/icon.png    |  Bin 9058 -> 0 bytes
 .../plugins/steps/carbonslicemerger/plugin.xml  |    1 -
 .../steps/carbonsortkeyandgroupby/icon.png      |  Bin 9058 -> 0 bytes
 .../steps/carbonsortkeyandgroupby/plugin.xml    |    1 -
 .../.kettle/plugins/steps/mdkeygenstep/icon.png |  Bin 9058 -> 0 bytes
 .../plugins/steps/mdkeygenstep/plugin.xml       |    1 -
 .../.kettle/plugins/steps/sortkeystep/icon.png  |  Bin 9058 -> 0 bytes
 .../plugins/steps/sortkeystep/plugin.xml        |    1 -
 .../org/carbondata/lcm/locks/LocalFileLock.java |    4 +-
 .../org/carbondata/lcm/locks/LockUsage.java     |    3 +-
 .../lcm/status/SegmentStatusManager.java        |  111 +-
 .../api/dataloader/DataLoadModel.java           |    9 +
 .../processing/csvload/DataGraphExecuter.java   |    8 +
 .../csvreaderstep/BlockDataHandler.java         |   83 +-
 .../processing/csvreaderstep/CsvInput.java      |    4 +-
 .../processing/csvreaderstep/CsvInputData.java  |    2 +
 .../processing/csvreaderstep/CsvInputMeta.java  |   21 +-
 .../dataprocessor/DataProcessTaskStatus.java    |   10 +
 .../dataprocessor/IDataProcessStatus.java       |    2 +
 .../graphgenerator/GraphGenerator.java          |    6 +
 .../configuration/GraphConfigurationInfo.java   |   10 +
 .../processing/mdkeygen/MDKeyGenStep.java       |   41 +-
 .../merger/step/CarbonSliceMergerStep.java      |    3 +-
 .../processing/schema/metadata/ColumnsInfo.java |   10 +
 .../store/CarbonFactDataHandlerColumnar.java    |  111 +-
 .../store/CarbonFactDataHandlerModel.java       |   63 +-
 .../store/SingleThreadFinalSortFilesMerger.java |    6 +-
 .../store/colgroup/ColGroupDataHolder.java      |    4 +-
 .../store/colgroup/ColGroupMinMax.java          |   30 +-
 .../store/writer/AbstractFactDataWriter.java    |  143 +-
 ...actDataWriterImplForIntIndexAndAggBlock.java |    6 +-
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      |   36 +
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |   19 +-
 .../FileStoreSurrogateKeyGenForCSV.java         |   12 +-
 .../processing/util/CarbonSchemaParser.java     |   44 +
 .../store/colgroup/ColGroupMinMaxTest.java      |  385 +--
 250 files changed, 8623 insertions(+), 7448 deletions(-)
----------------------------------------------------------------------



[41/50] [abbrv] incubator-carbondata git commit: Merge pull request #783 from HuaweiBigData/new-github

Posted by ch...@apache.org.
Merge pull request #783 from HuaweiBigData/new-github

update new github address info

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

Branch: refs/heads/master
Commit: d44c9e70cafa598e2e6a4ba35fb355eb8a6171bc
Parents: 167d527 6e3e4c3
Author: Liang Chen <ch...@apache.org>
Authored: Thu Jun 30 06:52:51 2016 +0530
Committer: GitHub <no...@github.com>
Committed: Thu Jun 30 06:52:51 2016 +0530

----------------------------------------------------------------------
 README.md | 9 +--------
 1 file changed, 1 insertion(+), 8 deletions(-)
----------------------------------------------------------------------



[16/50] [abbrv] incubator-carbondata git commit: Added Validation for delete segments query, load start time parameter (#752)

Posted by ch...@apache.org.
Added Validation for delete segments query, load start time parameter (#752)

* Added Validation for delete segments, load start time parameter

* fixed review comment


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

Branch: refs/heads/master
Commit: 73975b02ce8a8e62ad052c08ed712a32f1e02f37
Parents: 148285d
Author: Manu <ma...@gmail.com>
Authored: Sun Jun 26 10:30:53 2016 +0530
Committer: Ravindra Pesala <ra...@gmail.com>
Committed: Sun Jun 26 10:30:53 2016 +0530

----------------------------------------------------------------------
 .../core/load/LoadMetadataDetails.java          | 43 ++++++++++++++++++++
 .../execution/command/carbonTableSchema.scala   | 17 ++++++--
 .../dataretention/DataRetentionTestCase.scala   | 37 +++++++++++++++++
 .../lcm/status/SegmentStatusManager.java        | 33 +++++----------
 4 files changed, 103 insertions(+), 27 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/73975b02/core/src/main/java/org/carbondata/core/load/LoadMetadataDetails.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/load/LoadMetadataDetails.java b/core/src/main/java/org/carbondata/core/load/LoadMetadataDetails.java
index 41edd98..0250b2e 100644
--- a/core/src/main/java/org/carbondata/core/load/LoadMetadataDetails.java
+++ b/core/src/main/java/org/carbondata/core/load/LoadMetadataDetails.java
@@ -20,6 +20,13 @@
 package org.carbondata.core.load;
 
 import java.io.Serializable;
+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.constants.CarbonCommonConstants;
 
 public class LoadMetadataDetails implements Serializable {
 
@@ -28,6 +35,15 @@ public class LoadMetadataDetails implements Serializable {
   private String loadStatus;
   private String loadName;
   private String partitionCount;
+
+  /**
+   * LOGGER
+   */
+  private static final LogService LOGGER =
+      LogServiceFactory.getLogService(LoadMetadataDetails.class.getName());
+
+  private static final SimpleDateFormat parser =
+      new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
   /**
    * Segment modification or deletion time stamp
    */
@@ -126,6 +142,33 @@ public class LoadMetadataDetails implements Serializable {
   }
 
   /**
+   * return loadStartTime
+   * @return
+   */
+  public long getLoadStartTimeAsLong() {
+    return getTimeStamp(loadStartTime);
+  }
+
+  /**
+   * returns load start time as long value
+   * @param loadStartTime
+   * @return
+   */
+  private Long getTimeStamp(String loadStartTime) {
+    if (loadStartTime.isEmpty()) {
+      return null;
+    }
+
+    Date dateToStr = null;
+    try {
+      dateToStr = parser.parse(loadStartTime);
+      return dateToStr.getTime() * 1000;
+    } catch (ParseException e) {
+      LOGGER.error("Cannot convert" + loadStartTime + " to Time/Long type value" + e.getMessage());
+      return null;
+    }
+  }
+  /**
    * @param loadStartTime
    */
   public void setLoadStartTime(String loadStartTime) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/73975b02/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 59c8566..99cb9ca 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
@@ -20,7 +20,7 @@ package org.apache.spark.sql.execution.command
 import java.io.File
 import java.text.SimpleDateFormat
 import java.util
-import java.util.UUID
+import java.util.{Date, UUID}
 
 import scala.collection.JavaConverters._
 import scala.collection.mutable.ArrayBuffer
@@ -29,7 +29,10 @@ import scala.util.Random
 
 import org.apache.spark.SparkEnv
 import org.apache.spark.sql._
-import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference}
+import org.apache.spark.sql.catalyst.analysis.TypeCheckResult
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, Cast, Literal}
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.catalyst.util.DateTimeUtils.SQLTimestamp
 import org.apache.spark.sql.execution.{RunnableCommand, SparkPlan}
 import org.apache.spark.sql.hive.HiveContext
 import org.apache.spark.sql.types.TimestampType
@@ -1376,6 +1379,12 @@ private[sql] case class DeleteLoadsByLoadDate(
       sys.error(s"Table $schemaName.$tableName does not exist")
     }
 
+    val timeObj = Cast(Literal(loadDate), TimestampType).eval()
+    if(null == timeObj) {
+      val errorMessage = "Error: Invalid load start time format " + loadDate
+      throw new MalformedCarbonCommandException(errorMessage)
+    }
+
     var carbonTable = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
       .getCarbonTable(schemaName + '_' + tableName)
     var segmentStatusManager = new SegmentStatusManager(carbonTable.getAbsoluteTableIdentifier)
@@ -1389,8 +1398,8 @@ private[sql] case class DeleteLoadsByLoadDate(
     }
     var path = carbonTable.getMetaDataFilepath()
 
-
-    var invalidLoadTimestamps = segmentStatusManager.updateDeletionStatus(loadDate, path).asScala
+    var invalidLoadTimestamps = segmentStatusManager
+      .updateDeletionStatus(loadDate, path, timeObj.asInstanceOf[java.lang.Long]).asScala
     LOGGER.audit("Delete load by load date is successfull.")
     Seq.empty
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/73975b02/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 c749a7e..683dcf8 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
@@ -185,5 +185,42 @@ class DataRetentionTestCase extends QueryTest with BeforeAndAfterAll {
       sql("select count(*) from caRbon_TabLe_1"), Seq(Row(0)))
 
   }
+  test("RetentionTest_DeleteSegmentsByLoadTimeValiadtion") {
+
+    try {
+      sql(
+        "DELETE SEGMENTS FROM TABLE dataretentionTable where STARTTIME before" +
+        " 'abcd-01-01 00:00:00'")
+      assert(false)
+    } catch {
+      case e: MalformedCarbonCommandException =>
+        assert(e.getMessage.contains("Invalid load start time format"))
+      case _ => assert(false)
+    }
+
+    try {
+      sql(
+        "DELETE SEGMENTS FROM TABLE dataretentionTable where STARTTIME before" +
+        " '2099:01:01 00:00:00'")
+      assert(false)
+    } catch {
+      case e: MalformedCarbonCommandException =>
+        assert(e.getMessage.contains("Invalid load start time format"))
+      case _ => assert(false)
+    }
+
+    checkAnswer(
+      sql("SELECT country, count(salary) AS amount FROM DataRetentionTable WHERE country" +
+          " IN ('china','ind','aus','eng') GROUP BY country"
+      ),
+      Seq(Row("ind", 9))
+    )
+    sql("DELETE SEGMENTS FROM TABLE dataretentionTable where STARTTIME before '2099-01-01'")
+    checkAnswer(
+      sql("SELECT country, count(salary) AS amount FROM DataRetentionTable WHERE country" +
+          " IN ('china','ind','aus','eng') GROUP BY country"), Seq())
+
+
+  }
 
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/73975b02/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 8e4c627..8708779 100644
--- a/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
+++ b/processing/src/main/java/org/carbondata/lcm/status/SegmentStatusManager.java
@@ -19,7 +19,6 @@
 package org.carbondata.lcm.status;
 
 import java.io.*;
-import java.text.ParseException;
 import java.text.SimpleDateFormat;
 import java.util.ArrayList;
 import java.util.Arrays;
@@ -221,23 +220,9 @@ public class SegmentStatusManager {
    * @return -1 if first arg is less than second arg, 1 if first arg is greater than second arg,
    * 0 otherwise
    */
-  private Integer compareDateStrings(String loadValue, String userValue) {
-    SimpleDateFormat sdf = new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP);
-    SimpleDateFormat defaultSdf =
-        new SimpleDateFormat(CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT);
-    try {
-      Date loadDate = sdf.parse(loadValue);
-      Date userDate = defaultSdf.parse(userValue);
-      if (loadDate.before(userDate)) {
-        return -1;
-      } else if (loadDate.after(userDate)) {
-        return 1;
-      }
-      return 0;
+  private Integer compareDateValues(Long loadValue, Long userValue) {
 
-    } catch (ParseException pe) {
-      return null;
-    }
+    return loadValue.compareTo(userValue);
   }
 
   /**
@@ -296,7 +281,8 @@ public class SegmentStatusManager {
    * @param tableFolderPath
    * @return
    */
-  public List<String> updateDeletionStatus(String loadDate, String tableFolderPath) {
+  public List<String> updateDeletionStatus(String loadDate, String tableFolderPath,
+      Long loadStartTime) {
     ICarbonLock carbonLock =
         CarbonLockFactory.getCarbonLockObj(tableFolderPath, LockUsage.METADATA_LOCK);
     List<String> invalidLoadTimestamps = new ArrayList<String>(0);
@@ -320,7 +306,7 @@ public class SegmentStatusManager {
         listOfLoadFolderDetailsArray = readLoadMetadata(tableFolderPath);
         if (listOfLoadFolderDetailsArray != null && listOfLoadFolderDetailsArray.length != 0) {
           updateDeletionStatus(loadDate, listOfLoadFolderDetailsArray,
-              invalidLoadTimestamps);
+              invalidLoadTimestamps, loadStartTime);
           if (!invalidLoadTimestamps.isEmpty()) {
             LOG.warn("Load doesnt exist or it is already deleted , LoadTimestamps-"
                 + invalidLoadTimestamps);
@@ -438,14 +424,15 @@ public class SegmentStatusManager {
    * @return invalidLoadTimestamps
    */
   public void updateDeletionStatus(String loadDate,
-      LoadMetadataDetails[] listOfLoadFolderDetailsArray, List<String> invalidLoadTimestamps) {
+      LoadMetadataDetails[] listOfLoadFolderDetailsArray, List<String> invalidLoadTimestamps,
+      Long loadStartTime) {
     // For each load timestamp loop through data and if the
     // required load timestamp is found then mark
     // the metadata as deleted.
     boolean loadFound = false;
-    String loadStartTime = "Load Start Time: ";
+    String loadStartTimeString = "Load Start Time: ";
     for (LoadMetadataDetails loadMetadata : listOfLoadFolderDetailsArray) {
-      Integer result = compareDateStrings(loadMetadata.getLoadStartTime(), loadDate);
+      Integer result = compareDateValues(loadMetadata.getLoadStartTimeAsLong(), loadStartTime);
       if (null == result) {
         invalidLoadTimestamps.add(loadDate);
       } else if (result < 0) {
@@ -454,7 +441,7 @@ public class SegmentStatusManager {
           loadMetadata.setLoadStatus(CarbonCommonConstants.MARKED_FOR_DELETE);
           loadMetadata.setModificationOrdeletionTimesStamp(readCurrentTime());
           LOG.info("Info: " +
-              loadStartTime + loadMetadata.getLoadStartTime() +
+              loadStartTimeString + loadMetadata.getLoadStartTime() +
               " Marked for Delete");
         } else {
           // it is already deleted . can not delete it again.


[29/50] [abbrv] incubator-carbondata git commit: [Bug]fix bug in handle multiple quotes (#761)

Posted by ch...@apache.org.
[Bug]fix bug in handle multiple quotes (#761)

When handle data with quotes,we need to follow the rules:
1)the cell should be enclosed in quote character
2)inside the data eache quote is escaped using escape character.

for example:
"""abc""" -> "abc" (escape char and quote is ")
\u201d\"i\"" ->"i" (escape char is\ ,quote is ")


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

Branch: refs/heads/master
Commit: cdfd08c81a7df47bd99f145e1119fcdb21a8df6c
Parents: 462eb39
Author: linyixin <li...@huawei.com>
Authored: Tue Jun 28 01:04:55 2016 +0800
Committer: Vimal-Das <vi...@gmail.com>
Committed: Mon Jun 27 10:04:55 2016 -0700

----------------------------------------------------------------------
 .../resources/datawithcomplexspecialchar.csv    | 151 +++++++++++++++++++
 .../test/resources/datawithspecialcharacter.csv |   1 +
 .../dataload/TestLoadDataWithHiveSyntax.scala   |  25 ++-
 .../csvreaderstep/BlockDataHandler.java         |  46 +++++-
 .../processing/csvreaderstep/CsvInput.java      |   1 +
 .../processing/csvreaderstep/CsvInputData.java  |   1 +
 6 files changed, 220 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cdfd08c8/integration/spark/src/test/resources/datawithcomplexspecialchar.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datawithcomplexspecialchar.csv b/integration/spark/src/test/resources/datawithcomplexspecialchar.csv
new file mode 100644
index 0000000..b5d1beb
--- /dev/null
+++ b/integration/spark/src/test/resources/datawithcomplexspecialchar.csv
@@ -0,0 +1,151 @@
+customer_id,124_string_level_province,numeric_level,date_level,Time_level,account_num,lname,fname,mi,address1,address2,address3,address4,city,country,customer_region_id,phone1,phone2,marital_status,yearly_income,gender,total_children,num_children_at_home,education,member_card,occupation,houseowner,num_cars_owned,fullname
+1,Oaxaca,9535837227,26-08-1961,08:08:08,87462024688,Nowmer,Sheri,A.,2433 Bailey Road,,,,Tlaxiaco,Mexico,30,271-555-9715,119-555-1969,M,$30K - $50K,F,4,2,Partial High School,Bronze,Skilled Manual,Y,4,Sheri Nowmer
+2,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87470586299,Whelply,Derrick,I.,2219 Dewing Avenue,,,,Sooke,Canada,101,211-555-7669,807-555-9033,S,$70K - $90K,M,1,0,Partial High School,Bronze,Professional,N,3,Derrick Whelply
+3,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87475757600,Derry,Jeanne,,7640 First Ave.,,,,Issaquah,USA,21,656-555-2272,221-555-2493,M,$50K - $70K,F,1,1,Bachelors Degree,Bronze,Professional,Y,2,Jeanne Derry
+4,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87500482201,Spence,Michael,J.,337 Tosca Way,,,,Burnaby,Canada,92,929-555-7279,272-555-2844,M,$10K - $30K,M,4,4,Partial High School,Normal,Skilled Manual,N,2,Michael Spence
+5,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87514054179,Gutierrez,Maya,,8668 Via Neruda,,,,Novato,USA,42,387-555-7172,260-555-6936,S,$30K - $50K,F,3,0,Partial College,Silver,Manual,N,3,Maya Gutierrez
+6,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87517782449,Damstra,Robert,F.,1619 Stillman Court,,,,Lynnwood,USA,75,922-555-5465,333-555-5915,S,$70K - $90K,F,3,0,Bachelors Degree,Bronze,Professional,Y,3,Robert Damstra
+7,Oaxaca,9535837227,26-08-1961,08:08:08,87521172800,Kanagaki,Rebecca,,2860 D Mt. Hood Circle,,,,Tlaxiaco,Mexico,30,515-555-6247,934-555-9211,M,$30K - $50K,F,2,1,Partial High School,Bronze,Manual,Y,3,Rebecca Kanagaki
+8,"esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF ",0,10-05-1951,00:00:00,87539744377,Brunner,Kim,H.,6064 Brodia Court,,,,San Andres,Mexico,106,411-555-6825,130-555-6818,M,$50K - $70K,M,2,2,Bachelors Degree,Bronze,Professional,Y,3,Kim Brunner
+9,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87544797658,Blumberg,Brenda,C.,7560 Trees Drive,,,,Richmond,Canada,90,815-555-3975,642-555-6483,M,$10K - $30K,M,5,3,Partial High School,Normal,Skilled Manual,Y,1,Brenda Blumberg
+10,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,87568712234,Stanz,Darren,M.,1019 Kenwal Rd.,,,,Lake Oswego,USA,64,847-555-5443,212-555-8635,S,$30K - $50K,M,4,0,Bachelors Degree,Golden,Management,N,4,Darren Stanz
+11,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87572821378,Murraiin,Jonathan,V.,5423 Camby Rd.,,,,La Mesa,USA,11,612-555-4878,747-555-6928,S,$50K - $70K,M,4,0,High School Degree,Bronze,Manual,N,2,Jonathan Murraiin
+12,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87579237222,Creek,Jewel,C.,1792 Belmont Rd.,,,,Chula Vista,USA,13,555-555-2714,228-555-5450,S,$30K - $50K,F,1,0,High School Degree,Bronze,Skilled Manual,N,3,Jewel Creek
+13,LonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmembernameLonglevelmember,0,10-05-1951,00:00:00,87587122917,Medina,Peggy,A.,3796 Keller Ridge,,,,Mexico City,Mexico,2,343-555-9778,785-555-2371,S,$30K - $50K,M,4,0,High School Degree,Bronze,Manual,N,4,Peggy Medina
+14,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87592626810,Rutledge,Bryan,K.,3074 Ardith Drive,,,,Lincoln Acres,USA,10,659-555-3160,640-555-5439,M,$50K - $70K,F,2,2,Bachelors Degree,Bronze,Management,Y,2,Bryan Rutledge
+15,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87597749829,Cavestany,Walter,G.,7987 Seawind Dr.,,,,Oak Bay,Canada,99,471-555-8853,560-555-4646,S,$90K - $110K,M,3,0,Graduate Degree,Bronze,Professional,Y,3,Walter Cavestany
+16,select * from scenario1_84gsinglecsv_updatedcolumns_test3;,1.00E+14,02-10-1969,20:20:20,87603285908,Planck,Peggy,M.,4864 San Carlos,,,,Camacho,Mexico,27,698-555-7603,986-555-9424,M,$50K - $70K,F,2,2,High School Degree,Bronze,Skilled Manual,Y,2,Peggy Planck
+17,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87625473141,Marshall,Brenda,S.,2687 Ridge Road,,,,Arcadia,USA,51,771-555-6581,929-555-7260,S,$90K - $110K,M,3,0,High School Degree,Bronze,Management,Y,3,Brenda Marshall
+18,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87637655735,Wolter,Daniel,P.,2473 Orchard Way,,,,Altadena,USA,50,121-555-3960,717-555-5324,M,$30K - $50K,M,3,2,Partial College,Bronze,Professional,Y,4,Daniel Wolter
+19,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87650814652,Collins,Dianne,,551 Rainier Dr,,,,Oakland,USA,37,217-555-1567,921-555-5446,S,$70K - $90K,F,1,0,High School Degree,Bronze,Professional,N,1,Dianne Collins
+20,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87653979700,Baker,Beverly,,591 Merriewood Drive,,,,Spring Valley,USA,15,617-555-7972,860-555-3536,S,$10K - $30K,F,2,0,Partial High School,Normal,Manual,Y,2,Beverly Baker
+21,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87663244009,Castillo,Pedro,,1579 Plaza Rosa,,,,Renton,USA,73,975-555-7865,187-555-2286,S,$30K - $50K,M,1,0,High School Degree,Normal,Manual,Y,2,Pedro Castillo
+22,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87675641200,Borges,Laurie,,1873 Lyon Circle,,,,Bellingham,USA,78,610-555-6158,921-555-6608,M,$30K - $50K,F,4,4,High School Degree,Golden,Skilled Manual,Y,4,Laurie Borges
+23,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87678398489,Wyro,Shauna,K.,3114 Notre Dame Ave.,,,,La Jolla,USA,33,444-555-1517,625-555-9824,S,$30K - $50K,F,2,0,High School Degree,Silver,Manual,N,3,Shauna Wyro
+24,"esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF ",0,10-05-1951,00:00:00,87681713700,Wyllie,Jacqueline,,6318 Marclair Dr.,,,,Santa Fe,Mexico,109,102-555-7439,721-555-7431,S,$30K - $50K,F,0,0,High School Degree,Bronze,Manual,N,3,Jacqueline Wyllie
+25,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87686740159,Conley,Lin,N.,7814 Milburn Dr.,,,,Haney,Canada,93,506-555-4589,333-555-7096,S,$70K - $90K,F,3,0,High School Degree,Bronze,Professional,N,2,Lin Conley
+26,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87691326700,Bernard,Jose,,3928 San Francisco,,,,Burbank,USA,53,425-555-3796,452-555-3332,S,$10K - $30K,M,0,0,Partial High School,Normal,Skilled Manual,N,1,Jose Bernard
+27,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87700161933,Macaluso,Charles,B.,5678 McElroy Court,,,,Anacortes,USA,79,852-555-8574,986-555-1624,M,$30K - $50K,F,1,1,High School Degree,Bronze,Manual,Y,3,Charles Macaluso
+28,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87709978818,Wood,Lois,M.,2046 Las Palmas,,,,Everett,USA,81,794-555-6410,467-555-9146,S,$10K - $30K,F,1,0,Partial High School,Silver,Manual,Y,1,Lois Wood
+29,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87717585563,Armstrong,Kevin,A.,608 Lincoln Dr.,,,,Oak Bay,Canada,99,583-555-4474,125-555-6067,S,$10K - $30K,F,2,0,Partial High School,Normal,Manual,N,1,Kevin Armstrong
+30,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,87718932032,Goldey,Cody,A.,7230 Berrellesa Street,,,,Milwaukie,USA,65,548-555-9939,529-555-3217,S,$50K - $70K,F,4,0,Bachelors Degree,Bronze,Professional,Y,3,Cody Goldey
+31,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87729650139,Nickell,Paula,B.,2144 San Rafael,,,,Concord,USA,46,598-555-1289,448-555-2424,M,$50K - $70K,F,4,3,Graduate Degree,Golden,Professional,Y,3,Paula Nickell
+32,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87738383412,Long,Eric,J.,1676 Greer Ave.,,,,Ladner,Canada,96,937-555-2299,326-555-4120,S,$10K - $30K,F,2,0,Partial High School,Normal,Manual,Y,2,Eric Long
+33,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87742803692,Solimena,Mary,K.,6843 San Simeon Dr.,,,,Santa Cruz,USA,40,111-555-1278,268-555-1956,M,$30K - $50K,F,2,2,High School Degree,Bronze,Skilled Manual,N,2,Mary Solimena
+34,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87767067000,Muenich,Nathan,,6629 Polson Circle,,,,La Mesa,USA,11,361-555-7656,957-555-9020,M,$50K - $70K,F,3,1,Bachelors Degree,Silver,Management,Y,2,Nathan Muenich
+35,Yucatan,256,17-04-1920,12:00:00,87771052718,Young,Ed,,Rt. 470 Box A,,,,Merida,Mexico,26,457-555-5263,261-555-9142,M,$30K - $50K,M,5,1,Partial College,Bronze,Skilled Manual,Y,3,Ed Young
+36,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,87788449525,Arnold,Donna,K.,7403 N. Broadway,,,,Oregon City,USA,68,969-555-4928,972-555-5835,S,$10K - $30K,M,1,0,Partial High School,Normal,Skilled Manual,N,1,Donna Arnold
+37,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87797676324,Cooper,Jennifer,K.,5734 Ashford Court,,,,Seattle,USA,18,188-555-1163,299-555-4585,S,$50K - $70K,M,0,0,Bachelors Degree,Bronze,Professional,N,3,Jennifer Cooper
+38,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87801286578,Petty,Peggy,J.,8684 Military East,,,,Arcadia,USA,51,722-555-8456,134-555-8906,S,$30K - $50K,M,1,0,Partial College,Bronze,Management,N,1,Peggy Petty
+39,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,87808394432,Olguin,Jessica,A.,7270 Pepperidge Way,,,,Lebanon,USA,72,203-555-6978,384-555-6285,M,$90K - $110K,F,4,2,Graduate Degree,Bronze,Professional,Y,2,Jessica Olguin
+40,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87814209461,Burchett,Phyllis,,475 Santa Maria,,,,Santa Cruz,USA,40,761-555-3899,719-555-7549,S,$110K - $130K,M,1,0,Partial College,Bronze,Professional,Y,2,Phyllis Burchett
+41,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,87818307900,Bechard,Howard,,1970 Napa Ct.,,,,W. Linn,USA,66,616-555-6966,443-555-9474,S,$110K - $130K,M,1,0,High School Degree,Bronze,Professional,N,2,Howard Bechard
+42,"esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF ",0,10-05-1951,00:00:00,87849287390,Carter,Doris,M.,8084 Cardiff Drive,,,,San Andres,Mexico,106,535-555-6174,562-555-5709,M,$70K - $90K,F,3,3,Bachelors Degree,Golden,Management,Y,3,Doris Carter
+43,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87855359911,Sharp,Juanita,D.,9833 Mt. Dias Blv.,,,,Burbank,USA,53,962-555-1952,196-555-4002,S,$50K - $70K,F,2,0,Bachelors Degree,Bronze,Professional,N,4,Juanita Sharp
+44,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87868926525,Brunner,Sandra,B.,6202 Seeno St.,,,,Bellflower,USA,55,904-555-8788,577-555-2523,S,$30K - $50K,M,1,0,High School Degree,Bronze,Skilled Manual,Y,2,Sandra Brunner
+45,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,87869548800,Staton,Ernest,,862 The Alameda,,,,Renton,USA,73,693-555-6852,235-555-8445,S,$50K - $70K,M,5,0,Bachelors Degree,Golden,Management,Y,2,Ernest Staton
+46,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87894418239,Sims,Rose,M.,7484 Roundtree Drive,,,,Mill Valley,USA,38,658-555-3316,639-555-5595,M,$70K - $90K,F,1,1,Graduate Degree,Silver,Professional,N,2,Rose Sims
+47,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87894944000,De Carlo,Lauretta,E.,5053 Loftus Road,,,,Oak Bay,Canada,99,708-555-3666,558-555-4802,M,$50K - $70K,M,0,0,Bachelors Degree,Bronze,Professional,N,1,Lauretta De Carlo
+48,select * from scenario1_84gsinglecsv_updatedcolumns_test3;,1.00E+14,02-10-1969,20:20:20,87901939153,Williams,Mary,R.,1930 Almondwood Drive,,,,Camacho,Mexico,27,696-555-7759,324-555-4238,M,$50K - $70K,M,0,0,Bachelors Degree,Bronze,Professional,N,3,Mary Williams
+49,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87902239561,Burke,Terri,L.,9752 Jeanne Circle,,,,Arcadia,USA,51,770-555-6738,266-555-2074,S,$30K - $50K,F,4,0,High School Degree,Bronze,Skilled Manual,N,2,Terri Burke
+50,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87910291552,Osborn,Audrey,H.,9539 Glenside Dr,,,,Altadena,USA,50,119-555-4116,715-555-5480,S,$110K - $130K,M,4,0,Partial High School,Bronze,Professional,Y,4,Audrey Osborn
+51,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87923139333,Binai,Brian,C.,7616 Honey Court,,,,Oakland,USA,37,216-555-1723,920-555-5602,M,$90K - $110K,F,4,2,Graduate Degree,Bronze,Professional,N,3,Brian Binai
+52,"esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF ",0,10-05-1951,00:00:00,87939313100,Lozada,Concepcion,,7657 H St.,,,,Tixapan,Mexico,107,727-555-1388,731-555-2295,S,$70K - $90K,F,3,0,Bachelors Degree,Silver,Professional,N,2,Concepcion Lozada
+53,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87952010600,Tays,Paul,,5988 Red Leaf,,,,Oak Bay,Canada,99,298-555-3541,409-555-6962,M,$70K - $90K,M,4,2,Graduate Degree,Normal,Management,Y,4,Paul Tays
+54,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,87954998600,Thorton,Cheryl,,8938 West Hookston Road,,,,Metchosin,Canada,104,832-555-1834,244-555-2284,S,$30K - $50K,F,0,0,Partial College,Bronze,Professional,Y,3,Cheryl Thorton
+55,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87961350083,Dumin,Gary,,181 Gainsborough Drive,,,,Grossmont,USA,17,313-555-9355,733-555-3320,M,$10K - $30K,M,3,2,Partial High School,Normal,Skilled Manual,Y,1,Gary Dumin
+56,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87965223193,Chin,Pat,M.,3385 Crestview Drive,,,,Imperial Beach,USA,14,871-555-6277,829-555-9927,S,$10K - $30K,M,3,0,Partial High School,Normal,Manual,N,2,Pat Chin
+57,"esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF ",0,10-05-1951,00:00:00,87978752300,Lovell,Zach,,6912 Laguna St.,,,,San Andres,Mexico,106,375-555-3427,201-555-5934,S,$10K - $30K,M,3,0,Partial High School,Golden,Manual,Y,0,Zach Lovell
+58,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,87986992001,Ratcliff,Dave,A.,371 Ahneita Dr.,,,,Oakland,USA,37,294-555-2634,560-555-5827,S,$30K - $50K,F,3,0,Partial High School,Bronze,Skilled Manual,N,3,Dave Ratcliff
+59,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88010123400,Moss,Elizabeth,,4776 Kentucky Drive,,,,Burnaby,Canada,92,721-555-7412,855-555-9462,M,$30K - $50K,M,5,5,High School Degree,Bronze,Manual,Y,3,Elizabeth Moss
+60,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88018041300,Anderson,Lori,M.,1145 Paraiso Ct.,,,,Langley,Canada,94,663-555-5248,336-555-7984,S,$30K - $50K,M,3,0,Partial College,Silver,Skilled Manual,Y,3,Lori Anderson
+61,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88049879074,Everson,Michael,A.,3148 Rose Street,,,,Tacoma,USA,84,451-555-3312,894-555-4905,M,$70K - $90K,M,1,0,Bachelors Degree,Bronze,Professional,N,1,Michael Everson
+62,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88067190100,Borden,Mary,A.,2427 Notre Dame Ave.,,,,Santa Monica,USA,49,656-555-3434,636-555-5712,M,$30K - $50K,M,1,1,High School Degree,Golden,Skilled Manual,Y,2,Mary Borden
+63,Oaxaca,9535837227,26-08-1961,08:08:08,88081979800,Willson,Sue,,7340 Green St.,,,,Tlaxiaco,Mexico,30,467-555-9127,317-555-1262,S,$70K - $90K,F,4,0,Bachelors Degree,Bronze,Professional,N,2,Sue Willson
+64,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88084045052,Harris,Clayton,J.,6872 Thornwood Dr.,,,,Beaverton,USA,67,814-555-9899,203-555-2721,S,$10K - $30K,M,4,0,Partial High School,Normal,Manual,Y,2,Clayton Harris
+65,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88096334934,Contreras,Whitney,L.,2040 Encino Drive,,,,Metchosin,Canada,104,648-555-5221,145-555-9556,M,$90K - $110K,F,0,0,Partial College,Normal,Management,Y,3,Whitney Contreras
+66,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88112406730,Pusedu,Boyd,A.,1826 Corte Del Sol,,,,Metchosin,Canada,104,898-555-2599,833-555-7621,M,$110K - $130K,F,4,4,Partial College,Silver,Professional,Y,2,Boyd Pusedu
+67,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88115072846,Stedman,Cian,F.,9903 East Leland,,,,Westminster,Canada,91,994-555-9206,799-555-4085,M,$30K - $50K,M,3,1,Partial College,Bronze,Skilled Manual,Y,3,Cian Stedman
+68,select * from scenario1_84gsinglecsv_updatedcolumns_test3;,1.00E+14,02-10-1969,20:20:20,88118441700,Zocchi,Michele,A.,9943 Stonehedge Dr.,,,,Hidalgo,Mexico,25,606-555-8871,610-555-9778,S,$30K - $50K,F,0,0,High School Degree,Silver,Manual,Y,1,Michele Zocchi
+69,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88137775063,Walsh,Jean,M.,931 Corte De Luna,,,,Los Angeles,USA,3,725-555-5106,837-555-8528,M,$10K - $30K,M,1,1,Partial High School,Normal,Skilled Manual,Y,1,Jean Walsh
+70,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88144207000,Renfro,Leopoldo,E.,1226 Shoenic,,,,Coronado,USA,8,360-555-3399,910-555-7506,M,$30K - $50K,M,0,0,High School Degree,Bronze,Manual,Y,1,Leopoldo Renfro
+71,Sinaloa,1.00E+78,08-04-1967,18:18:18,88161163926,Brockett,Donna,A.,2467 Clearland Circle,,,,La Cruz,Mexico,29,741-555-1921,260-555-4885,S,$10K - $30K,M,5,0,Partial High School,Normal,Manual,N,1,Donna Brockett
+72,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88161249756,Anderson,Laurie,A.,5671 Bel Air Dr.,,,,Shawnee,Canada,105,398-555-7842,356-555-2492,M,$10K - $30K,F,5,5,Partial High School,Normal,Skilled Manual,Y,2,Laurie Anderson
+73,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88168829679,Gomez,Louis,C.,7166 Brock Lane,,,,Yakima,USA,89,802-555-4992,629-555-7499,M,$30K - $50K,F,4,3,High School Degree,Golden,Skilled Manual,Y,3,Louis Gomez
+74,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88172026872,Walker,Julie,L.,3281 Hillview Dr.,,,,Ballard,USA,20,722-555-4199,987-555-7392,S,$50K - $70K,M,3,0,Bachelors Degree,Bronze,Professional,Y,3,Julie Walker
+75,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88182830810,Jones,Jay,W.,5030 Blue Ridge Dr.,,,,Lynnwood,USA,75,248-555-8978,383-555-2028,S,$30K - $50K,F,3,0,High School Degree,Bronze,Manual,N,3,Jay Jones
+76,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88185078501,Winfrey,Gayle,J.,1399 Firestone Drive,,,,Port Orchard,USA,77,191-555-6813,764-555-9549,S,$70K - $90K,F,1,0,High School Degree,Normal,Professional,N,1,Gayle Winfrey
+77,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88203880800,Brown,Yasmina,,6058 Hill Street,#4,,,Downey,USA,58,569-555-5452,351-555-1703,S,$50K - $70K,M,2,0,Bachelors Degree,Normal,Professional,Y,4,Yasmina Brown
+78,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88243577480,Touchstone,Joseph,A. C.,2681 Eagle Peak,,,,Salem,USA,23,774-555-5574,754-555-7853,M,$10K - $30K,M,1,1,Partial High School,Silver,Manual,Y,1,Joseph Touchstone
+79,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88249907493,Bakhtyari,Mary,J.,7594 Alexander Pl.,,,,Tacoma,USA,84,585-555-2267,435-555-3403,S,$70K - $90K,M,2,0,Bachelors Degree,Bronze,Management,Y,4,Mary Bakhtyari
+80,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88262524604,Glass,Melvin,W.,7126 Edie Ct.,,,,Lemon Grove,USA,12,812-555-1017,200-555-2838,S,$70K - $90K,F,1,0,Bachelors Degree,Bronze,Management,Y,4,Melvin Glass
+81,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88265968561,Cohen,Kristin,S.,2294 West 39th St.,,,,Milwaukie,USA,65,646-555-5338,143-555-9674,S,$10K - $30K,M,2,0,Partial High School,Normal,Skilled Manual,Y,2,Kristin Cohen
+82,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88275710344,Kharman,Susan,,2080 Sycamore Drive,,,,Richmond,USA,35,896-555-2717,831-555-7738,S,$10K - $30K,F,4,0,Partial High School,Normal,Skilled Manual,Y,1,Susan Kharman
+83,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88284432888,Kirschner,Gordon,R.,158 Walnut Ave,,,,Arcadia,USA,51,331-555-3981,796-555-4203,S,$110K - $130K,F,3,0,Partial High School,Golden,Management,Y,2,Gordon Kirschner
+84,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88304206132,Burger,William,W.,2854 Magda Way,,,,Haney,Canada,93,604-555-8988,608-555-9895,S,$50K - $70K,M,4,0,Bachelors Degree,Bronze,Professional,N,3,William Burger
+85,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88346847891,Blau,Lana,L.,1185 Dallis Drive,,,,Novato,USA,42,723-555-5224,835-555-8645,M,$30K - $50K,M,3,3,High School Degree,Golden,Skilled Manual,Y,3,Lana Blau
+86,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88352523400,Derby,Donna,,8154 Via Mexico,,,,Bremerton,USA,76,358-555-3517,908-555-7624,S,$30K - $50K,M,4,0,Partial High School,Normal,Skilled Manual,Y,4,Donna Derby
+87,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88355388045,Thomson,Kathleen,K.,6740 Jalisco,,,,N. Vancouver,Canada,97,739-555-2038,258-555-5003,S,$70K - $90K,F,4,0,High School Degree,Bronze,Management,N,3,Kathleen Thomson
+88,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88380010700,Walton,Gabriel,,2601 Cambridge Drive,,,,Milwaukie,USA,65,396-555-7960,466-555-4870,S,$30K - $50K,F,2,0,High School Degree,Bronze,Manual,N,4,Gabriel Walton
+89,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88386346101,Meastas,Bishop,,4096 San Remo,,,,Woodland Hills,USA,62,912-555-7369,739-555-9877,S,$70K - $90K,F,4,0,High School Degree,Bronze,Management,N,2,Bishop Meastas
+90,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88386882543,Kouba,Geneva,,7554 Lakeview Pl.,,,,Sooke,Canada,101,832-555-6577,197-555-9770,M,$70K - $90K,F,2,2,High School Degree,Bronze,Professional,Y,3,Geneva Kouba
+91,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88397842049,Clark,Tricia,F.,1960 Via Catanzaro,,,,Long Beach,USA,48,597-555-6013,732-555-8063,M,$50K - $70K,M,1,1,Bachelors Degree,Bronze,Professional,N,1,Tricia Clark
+92,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88398657441,Munn,Debi,,5672 Hale Dr.,,,,Arcadia,USA,51,540-555-3848,874-555-2927,M,$30K - $50K,M,4,2,High School Degree,Bronze,Skilled Manual,Y,4,Debi Munn
+93,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88416295647,Goldberg,Patricia,,332 Laguna Niguel,,,,Woodburn,USA,69,989-555-7255,770-555-3505,M,$130K - $150K,M,5,4,Bachelors Degree,Golden,Professional,Y,3,Patricia Goldberg
+94,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88416842818,Magenheim,Susan,L.,6954 Garcia Ranch Road,,,,Vancouver,Canada,5,293-555-7377,274-555-9655,S,$10K - $30K,M,1,0,Partial High School,Normal,Skilled Manual,Y,1,Susan Magenheim
+95,Oaxaca,9535837227,26-08-1961,08:08:08,88426868915,Hance,Nancy,D.,4523 Via Cortez,,,,Tlaxiaco,Mexico,30,471-555-9125,320-555-1260,S,$130K - $150K,F,2,0,Graduate Degree,Silver,Professional,Y,2,Nancy Hance
+96,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88445172309,Mehlert,Rhonda,L.,1400 Gibrix Drive,,,,Beaverton,USA,67,698-555-7875,986-555-9696,S,$50K - $70K,M,0,0,Bachelors Degree,Normal,Management,Y,2,Rhonda Mehlert
+97,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88470690800,Duran,Paula,,9222 Roseann Drive,#59,,,Richmond,Canada,90,532-555-3196,928-555-7532,M,$50K - $70K,M,4,4,High School Degree,Golden,Skilled Manual,Y,2,Paula Duran
+98,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88486376400,Earley,Margaret,,9009 W 46th St,,,,Richmond,Canada,90,121-555-4232,717-555-5596,M,$130K - $150K,F,1,1,High School Degree,Bronze,Management,Y,3,Margaret Earley
+99,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88506181836,Horne,Elizabeth,,7086 O St.,,,,Port Orchard,USA,77,217-555-1839,682-555-2060,S,$30K - $50K,F,3,0,Bachelors Degree,Bronze,Professional,N,2,Elizabeth Horne
+100,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88506991900,Hunt,Mary,,7127 Morengo Court,,,,San Jose,USA,39,490-555-6846,493-555-7753,M,$90K - $110K,F,2,2,Partial High School,Silver,Professional,Y,3,Mary Hunt
+101,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88507222533,Allen,Libby,,5458 Encino Drive,,,,N. Vancouver,Canada,97,848-555-6739,721-555-6503,S,$10K - $30K,F,1,0,Partial High School,Normal,Manual,N,1,Libby Allen
+102,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88530171513,Bruno,Bonnie,F.,8408 Via Del Sol,,,,Langford,Canada,102,244-555-1375,794-555-5482,M,$30K - $50K,M,2,0,Bachelors Degree,Bronze,Professional,N,4,Bonnie Bruno
+103,"""state province # 124""",4124,26-01-1950,16:16:16,88531920313,Bowers,Angela,W.,9650 Valley View Road,,,,Acapulco,Mexico,28,737-555-2156,256-555-5120,M,$50K - $70K,M,5,3,Bachelors Degree,Golden,Professional,Y,2,Angela Bowers
+104,select * from scenario1_84gsinglecsv_updatedcolumns_test3;,1.00E+14,02-10-1969,20:20:20,88537885546,Gonzales,Donald,C.,2855 Playa,,,,Hidalgo,Mexico,25,394-555-8077,352-555-2727,M,$30K - $50K,F,1,1,Partial College,Bronze,Clerical,N,2,Donald Gonzales
+105,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88539907932,Valdez,Sylvester,,4350 Mota Dr.,,,,National City,USA,9,137-555-8885,625-555-7734,M,$150K +,M,3,3,Bachelors Degree,Golden,Management,Y,3,Sylvester Valdez
+106,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88556258678,Stewart,John,C.,7808 Brown St.,,,,Long Beach,USA,48,717-555-4435,983-555-7627,S,$50K - $70K,M,5,0,Bachelors Degree,Bronze,Professional,Y,2,John Stewart
+107,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88589957952,Velasquez,Barney,C.,2214 Solano,,,,Cliffside,Canada,103,483-555-3870,618-555-5920,M,$30K - $50K,M,2,2,High School Degree,Bronze,Manual,Y,4,Barney Velasquez
+108,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88593557476,Dubois,Kenneth,R.,8582 San Jose Ave.,,,,Shawnee,Canada,105,425-555-1706,998-555-4442,M,$90K - $110K,M,1,1,Bachelors Degree,Bronze,Management,Y,3,Kenneth Dubois
+109,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88597859740,Maestas,Kate,A.,3241 Brock Lane,,,,Newton,Canada,95,214-555-8770,656-555-1363,S,$50K - $70K,M,2,0,Bachelors Degree,Bronze,Professional,N,2,Kate Maestas
+110,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88604978322,Underwood,Josie,B.,9863 Ridge Place,,,,West Covina,USA,60,179-555-5235,160-555-7513,S,$10K - $30K,M,3,0,Partial High School,Normal,Manual,N,1,Josie Underwood
+111,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88620250821,Strain,Ramon,,4777 Rockne Drive,,,,Palo Alto,USA,41,890-555-1927,979-555-6720,M,$70K - $90K,F,2,2,Bachelors Degree,Bronze,Management,Y,2,Ramon Strain
+112,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88622546792,Azari,Pat,C.,4310 Kenston Dr.,,,,Santa Monica,USA,49,217-555-9677,506-555-2499,M,$30K - $50K,F,4,1,High School Degree,Bronze,Skilled Manual,Y,3,Pat Azari
+113,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88624070286,Dabit,Bob,,9476 Hill Top Rd.,,,,Sooke,Canada,101,291-555-8656,448-555-9335,M,$110K - $130K,F,2,1,Bachelors Degree,Bronze,Professional,Y,3,Bob Dabit
+114,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88631725311,Carmona,Marty,T.,9263 Mt. McKinley Ct.,,,,Sooke,Canada,101,541-555-6035,237-555-7399,S,$150K +,F,3,0,High School Degree,Silver,Professional,Y,2,Marty Carmona
+115,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88638226985,Saxton,Gina,M.,620 Woodside Ct.,,,,Altadena,USA,50,330-555-4138,795-555-4359,S,$90K - $110K,F,3,0,Bachelors Degree,Silver,Professional,Y,2,Gina Saxton
+116,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88648338913,McLaughlin,Juan,R.,661 Miller Ave.,,,,Burnaby,Canada,92,602-555-9145,845-555-4709,S,$90K - $110K,M,2,0,High School Degree,Bronze,Management,N,3,Juan McLaughlin
+117,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88656943440,Eyster,Carol,,1648 Eastgate Lane,,,,Albany,USA,71,960-555-9038,833-555-8802,M,$90K - $110K,F,3,1,Graduate Degree,Bronze,Professional,Y,3,Carol Eyster
+118,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88662506341,Henry,Nancy,J.,4598 Manila Avenue,,,,Bremerton,USA,76,356-555-3673,906-555-7780,S,$50K - $70K,M,2,0,Bachelors Degree,Bronze,Professional,Y,2,Nancy Henry
+119,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88665478229,Barros,Chris,C.,3183 Trasher Road,,,,N. Vancouver,Canada,97,737-555-2195,256-555-5159,S,$10K - $30K,M,4,0,Partial High School,Normal,Manual,Y,1,Chris Barros
+120,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88673519492,Groome,Christopher,J.,6387 Scenic Avenue,,,,Milwaukie,USA,65,634-555-2773,353-555-2766,M,$30K - $50K,F,0,0,High School Degree,Bronze,Skilled Manual,Y,4,Christopher Groome
+121,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88683427572,Hill,Anna,L.,7883 Mitchell Canyon Court,,,,Santa Monica,USA,49,137-555-8923,864-555-2430,S,$30K - $50K,F,5,0,High School Degree,Silver,Manual,Y,2,Anna Hill
+122,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88690803647,Williams,Ramon,N.,3997 Via De Luna,,,,Walla Walla,USA,88,718-555-4473,983-555-7666,M,$30K - $50K,F,4,1,High School Degree,Bronze,Skilled Manual,N,3,Ramon Williams
+123,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88708876371,Masters,Scott,,5747 Shirley Drive,,,,Richmond,USA,35,483-555-3909,618-555-5959,M,$30K - $50K,F,2,0,High School Degree,Bronze,Manual,Y,1,Scott Masters
+124,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88719449639,"Brady, Jr.",Joseph,G.,2115 Pasado,,,,Oakland,USA,37,426-555-1745,999-555-4480,S,$70K - $90K,F,5,0,Bachelors Degree,Bronze,Professional,N,2,"Joseph Brady, Jr."
+125,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88734051585,Gray,Ellen,M.,6774 Bonanza,,,,Bellflower,USA,55,214-555-8809,656-555-1402,M,$70K - $90K,M,2,2,Bachelors Degree,Bronze,Professional,Y,3,Ellen Gray
+126,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88735752105,Fielder,Henry,T.,3397 Rancho View Drive,,,,Ballard,USA,20,179-555-5273,160-555-7552,S,$110K - $130K,F,2,0,Partial College,Normal,Professional,Y,2,Henry Fielder
+127,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88744179606,Finnell,Jeanine,L.,8310 Ridge Circle,,,,Everett,USA,81,103-555-4226,192-555-9019,S,$10K - $30K,M,1,0,Partial High School,Normal,Skilled Manual,Y,0,Jeanine Finnell
+128,1 2 3 4 5 6 7 8 9 0 ~!@#$%^&*() some thing long OR,0,10-05-1951,00:00:00,88750257492,Schuetz,Ian,T.,7842 Ygnacio Valley Road,,,,Salem,USA,23,330-555-2976,618-555-4797,S,$30K - $50K,F,1,0,High School Degree,Bronze,Skilled Manual,N,4,Ian Schuetz
+129,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88752945065,Runyon,Richard,,5666 Hazelnut Lane,,,,Concord,USA,46,403-555-1955,561-555-2633,M,$50K - $70K,F,0,0,Bachelors Degree,Bronze,Professional,Y,3,Richard Runyon
+130,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88771945834,Potter,Linda,M.,5452 Corte Gilberto,,,,Concord,USA,46,653-555-8333,349-555-9697,S,$110K - $130K,F,4,0,Graduate Degree,Bronze,Professional,N,2,Linda Potter
+131,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88772315979,Smith,Mary,E.,874 Olivera Road,,,,La Jolla,USA,33,749-555-5940,315-555-6162,S,$10K - $30K,M,3,0,Partial High School,Normal,Manual,Y,2,Mary Smith
+132,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88781231641,Brumagen,Barbara,A.,3570 Court Lane,,,,Cliffside,Canada,103,122-555-1947,365-555-6512,S,$30K - $50K,F,0,0,High School Degree,Silver,Skilled Manual,N,3,Barbara Brumagen
+133,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88788833022,Hamilton,Thelma,L.,1902 Santa Cruz,,,,Burbank,USA,53,480-555-1840,353-555-1604,M,$30K - $50K,M,1,0,High School Degree,Bronze,Skilled Manual,Y,3,Thelma Hamilton
+134,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88804722428,J Wilbert,Shirley,L.,4852 Chaparral Court,,,,Spokane,USA,87,115-555-9133,426-555-9583,S,$50K - $70K,M,4,0,Bachelors Degree,Bronze,Professional,Y,1,Shirley J Wilbert
+135,"esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF ",0,10-05-1951,00:00:00,88820129036,Ingram,Jeremy,T.,6093 Midway Ct.,,,,Tixapan,Mexico,107,257-555-3997,676-555-6962,M,$110K - $130K,M,5,0,Bachelors Degree,Silver,Professional,Y,3,Jeremy Ingram
+136,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88825504183,Haddix,Sharon,A.,9297 Kenston Dr.,,,,Lynnwood,USA,75,153-555-4576,772-555-4569,M,$150K +,M,3,3,Bachelors Degree,Golden,Professional,Y,3,Sharon Haddix
+137,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88846119642,Thomas,Leland,P.,793 Crawford Street,,,,Newport Beach,USA,59,557-555-1726,384-555-4233,M,$130K - $150K,F,3,3,Partial College,Bronze,Professional,Y,3,Leland Thomas
+138,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,88880816698,Wilson,Don,R.,4251 San Onofre Court,,,,Oak Bay,Canada,99,238-555-6276,503-555-9469,S,$90K - $110K,F,4,0,Partial High School,Silver,Professional,Y,2,Don Wilson
+139,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88896459341,Stovall,Wayne,,8656 Lakespring Place,,,,Coronado,USA,8,593-555-6287,728-555-8337,M,$30K - $50K,M,1,1,Partial High School,Bronze,Manual,Y,1,Wayne Stovall
+140,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,88966604471,Russell,Douglas,D.,5025 Holiday Hills,,,,La Mesa,USA,11,536-555-4122,209-555-6858,M,$10K - $30K,M,2,1,Partial High School,Normal,Skilled Manual,Y,1,Douglas Russell
+141,"esc  !@~##%%&**(*&((*()()*  ""  some thing ""  ' DF ",0,10-05-1951,00:00:00,88974345326,Doose,Maureen,,7028 San Gabriel Dr.,,,,Santa Anita,Mexico,108,324-555-2187,767-555-3779,M,$10K - $30K,M,4,3,Partial High School,Normal,Skilled Manual,N,1,Maureen Doose
+142,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,88980621695,Morreale,Ann,,6307 Grenadine Way,,,,Renton,USA,73,289-555-7651,270-555-9929,M,$50K - $70K,F,2,1,Bachelors Degree,Golden,Professional,Y,4,Ann Morreale
+143,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,89028214812,Rupert,Sheila,M.,1220 Bradford Way,,,,Glendale,USA,54,101-555-4344,190-555-9137,M,$30K - $50K,F,5,5,Partial College,Golden,Manual,Y,3,Sheila Rupert
+144,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,89028284549,Pierson,Ashley,E.,1999 Orangewood Court,,,,Westminster,Canada,91,328-555-3094,616-555-4915,S,$10K - $30K,M,3,0,Partial High School,Bronze,Skilled Manual,Y,0,Ashley Pierson
+145,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,89029260873,Nickels,Ivan,,9821 Juniper Drive,,,,Victoria,Canada,6,401-555-2072,559-555-2751,M,$10K - $30K,M,3,3,Partial High School,Normal,Manual,Y,2,Ivan Nickels
+146,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,89035692811,Neuhauser,Chad,D.,9608 Grammercy Lane,,,,Victoria,Canada,6,651-555-8451,347-555-9815,S,$130K - $150K,M,2,0,Partial High School,Bronze,Professional,Y,4,Chad Neuhauser
+147,https://www.google.co.in/?gws_rd=cr&ei=BF7rUqOoEc6GrgeIooHQDQ#q=India+state BC,1.51E+09,15-08-1947,07:14:54,89059253334,Coke,Ann,,7685 Imperial Dr,,,,Sooke,Canada,101,747-555-6058,551-555-9937,S,$110K - $130K,M,4,0,Bachelors Degree,Bronze,Professional,Y,2,Ann Coke
+148,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,89061329364,Scarbrough,Jon,M.,7726 Driftwood Drive,,,,Kirkland,USA,63,120-555-2065,363-555-6629,S,$70K - $90K,M,2,0,High School Degree,Bronze,Management,N,2,Jon Scarbrough
+149,SpecialCharacter level ~!@#$%^&*()_+| CA,0,10-05-1951,00:00:00,89073216915,Martensen,Lillian,F.,8713 Yosemite Ct.,,,,Chula Vista,USA,13,478-555-1958,590-555-5380,S,$50K - $70K,M,1,0,Bachelors Degree,Bronze,Professional,Y,1,Lillian Martensen
+150,&&  1 2 3 4 5 special set to test &  & &  _ _ _  786,1,01-01-1900,23:59:59,89090994596,McCormick,Mary,E.,9008 Creekside Drive,,,,Seattle,USA,18,113-555-9251,424-555-9701,M,$30K - $50K,F,4,1,High School Degree,Bronze,Manual,Y,3,Mary McCormick
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cdfd08c8/integration/spark/src/test/resources/datawithspecialcharacter.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/datawithspecialcharacter.csv b/integration/spark/src/test/resources/datawithspecialcharacter.csv
index f807120..12cc8fa 100644
--- a/integration/spark/src/test/resources/datawithspecialcharacter.csv
+++ b/integration/spark/src/test/resources/datawithspecialcharacter.csv
@@ -35,3 +35,4 @@ imei,specialchar
 1AA33,fslash/hi
 1AA34,a@b
 1AA35,\\n
+1AA36,"\"i\""

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cdfd08c8/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
index a75f020..1a6259a 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/dataload/TestLoadDataWithHiveSyntax.scala
@@ -257,7 +257,8 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
     ("ayush@b.com"),Row("ayushb.com")))
     sql("DROP TABLE IF EXISTS t3")
   }
-  test("test carbon table data loading with special character") {
+
+  test("test carbon table data loading with special character 1") {
     sql("DROP TABLE IF EXISTS t3")
 
     sql("""
@@ -267,12 +268,30 @@ class TestLoadDataWithHiveSyntax extends QueryTest with BeforeAndAfterAll {
 
     sql("""
        LOAD DATA LOCAL INPATH './src/test/resources/datawithspecialcharacter.csv' into table t3
-          options ('DELIMITER'=',', 'QUOTECHAR'='\"')
+          options ('DELIMITER'=',', 'QUOTECHAR'='"')
        """)
-    checkAnswer(sql("select count(*) from t3"), Seq(Row(36)))
+    checkAnswer(sql("select count(*) from t3"), Seq(Row(37)))
+    checkAnswer(sql("select specialchar from t3 where imei='1AA36'"),Seq(Row("\"i\"")))
     sql("DROP TABLE IF EXISTS t3")
   }
 
+  test("test carbon table data loading with special character 2") {
+    sql("DROP TABLE IF EXISTS t3")
+
+    sql("""
+        CREATE table t3(customer_id int, 124_string_level_province String, date_level String,
+        Time_level String, lname String, fname String, mi String, address1 String, address2 String, address3 String, address4 String, city String, country String, phone1 String, phone2 String, marital_status String, yearly_income String, gender String, education String, member_card String, occupation String, houseowner String, fullname String, numeric_level double, account_num double, customer_region_id int, total_children int, num_children_at_home int, num_cars_owned int)
+        STORED BY 'org.apache.carbondata.format'
+        """)
+
+    sql("""
+       LOAD DATA LOCAL INPATH './src/test/resources/datawithcomplexspecialchar.csv' into
+       table t3 options ('DELIMITER'=',', 'QUOTECHAR'='"','ESCAPECHAR'='"')
+        """)
+    checkAnswer(sql("select count(*) from t3"), Seq(Row(150)))
+    checkAnswer(sql("select 124_string_level_province from t3 where customer_id=103"),Seq(Row("\"state province # 124\"")))
+    sql("DROP TABLE IF EXISTS t3")
+  }
 
   override def afterAll {
     sql("drop table carbontable")

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cdfd08c8/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 4a56b85..e1201d7 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/BlockDataHandler.java
@@ -136,6 +136,31 @@ public class BlockDataHandler {
     return false;
   }
 
+  /**
+   * <pre>
+   * [abcd "" defg] --> [abcd " defg]
+   * [""""] --> [""]
+   * [""] --> ["]
+   * </pre>
+   *
+   * @return the byte array with escaped enclosures escaped.
+   */
+  public byte[] removeEscapedEnclosures(byte[] field, int nrEnclosuresFound) {
+    byte[] result = new byte[field.length - nrEnclosuresFound];
+    int resultIndex = 0;
+    for (int i = 0; i < field.length; i++) {
+      if (field[i] == data.enclosure[0]) {
+        if (!(i + 1 < field.length && field[i + 1] == data.enclosure[0])) {
+          // Not an escaped enclosure...
+          result[resultIndex++] = field[i];
+        }
+      } else {
+        result[resultIndex++] = field[i];
+      }
+    }
+    return result;
+  }
+
   public byte[] removeEscapeChar(byte[] field, byte[] escapeChar) {
     byte[] result = new byte[field.length];
     int resultIndex = 0;
@@ -298,6 +323,8 @@ public class BlockDataHandler {
         boolean enclosureFound = false;
         boolean quoteAfterDelimiter = false;
         boolean quoteBeforeDelimiterOrCrLf = false;
+        int escapedEnclosureFound = 0;
+        boolean escapedEnclosureFlag = false;
         while (!delimiterFound) {
           // If we find the first char, we might find others as well ;-)
           // Single byte delimiters only for now.
@@ -406,6 +433,10 @@ public class BlockDataHandler {
                       data.enclosure);
               if (!keepGoing) {
                 outOfEnclosureFlag = !outOfEnclosureFlag;
+                if(this.endBuffer -1 >= 0 && data.escapeCharMatcher.matchesPattern(this
+                    .byteBuffer,this.endBuffer - 1,data.escapeCharacter)){
+                  escapedEnclosureFlag = true;
+                }
                 // We found an enclosure character.
                 // Read another byte...
                 if (this.increaseEndBuffer()) {
@@ -422,6 +453,9 @@ public class BlockDataHandler {
                         data.enclosure);
                 if (keepGoing) {
                   outOfEnclosureFlag = !outOfEnclosureFlag;
+                  if (!escapedEnclosureFlag || data.enclosure[0] == data.escapeCharacter[0]) {
+                    escapedEnclosureFound++;
+                  }
                 } else {
                   /**
                    * <pre>
@@ -445,7 +479,11 @@ public class BlockDataHandler {
                 }
 
               }
-              if (!keepGoing) {
+              if (this.endBuffer - 1 >= 0
+                  && data.delimiterMatcher.matchesPattern(this.byteBuffer, this.endBuffer,
+                  data.delimiter)
+                  || data.crLfMatcher.isReturn(this.byteBuffer, this.endBuffer)
+                  || data.crLfMatcher.isLineFeed(this.byteBuffer, this.endBuffer)) {
                 if (data.enclosureMatcher
                     .matchesPattern(this.byteBuffer, this.endBuffer - 1, data.enclosure)) {
                   quoteBeforeDelimiterOrCrLf = true;
@@ -507,7 +545,11 @@ public class BlockDataHandler {
         byte[] field = new byte[length];
         System.arraycopy(this.byteBuffer, this.startBuffer, field, 0, length);
 
-        if(quoteAfterDelimiter && quoteBeforeDelimiterOrCrLf){
+        if (escapedEnclosureFound > 0 && quoteAfterDelimiter && quoteBeforeDelimiterOrCrLf) {
+          field = this.removeEscapedEnclosures(field, escapedEnclosureFound);
+        }
+
+        if(doConversions && quoteAfterDelimiter && data.escapeCharacter[0] != data.enclosure[0]){
           field = removeEscapeChar(field,data.escapeCharacter);
         }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cdfd08c8/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 42f20cc..a1a3618 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInput.java
@@ -569,6 +569,7 @@ public class CsvInput extends BaseStep implements StepInterface {
         }
       }
 
+      data.escapeCharMatcher = new SingleBytePatternMatcher();
       switch (data.encodingType) {
         case DOUBLE_BIG_ENDIAN:
           data.crLfMatcher = new MultiByteBigCrLfMatcher();

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cdfd08c8/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
----------------------------------------------------------------------
diff --git a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
index f97afed..d28b77b 100644
--- a/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
+++ b/processing/src/main/java/org/carbondata/processing/csvreaderstep/CsvInputData.java
@@ -45,6 +45,7 @@ public class CsvInputData extends BaseStepData implements StepDataInterface {
   public EncodingType encodingType;
   public PatternMatcherInterface delimiterMatcher;
   public PatternMatcherInterface enclosureMatcher;
+  public PatternMatcherInterface escapeCharMatcher;
   public CrLfMatcherInterface crLfMatcher;
 
   public CsvInputData() {


[48/50] [abbrv] incubator-carbondata git commit: Merge remote-tracking branch 'carbon_master/master' into apache/master

Posted by ch...@apache.org.
Merge remote-tracking branch 'carbon_master/master' into apache/master

Conflicts:
	README.md
	core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
	core/src/main/java/org/carbondata/core/keygenerator/directdictionary/timestamp/TimeStampDirectDictionaryGenerator.java
	core/src/main/java/org/carbondata/core/util/CarbonUtil.java
	core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
	core/src/main/java/org/carbondata/query/aggregator/impl/CustomAggregatorHelper.java
	core/src/main/java/org/carbondata/query/carbon/executor/internal/impl/InternalDetailQueryExecutor.java
	core/src/main/java/org/carbondata/query/carbon/processor/impl/AggregateQueryBlockProcessor.java
	core/src/main/java/org/carbondata/query/filter/executer/ColGroupFilterExecuterImpl.java
	core/src/main/java/org/carbondata/query/filter/executer/IncludeColGroupFilterExecuterImpl.java
	core/src/main/java/org/carbondata/scan/executor/util/QueryUtil.java
	core/src/main/java/org/carbondata/scan/expression/Expression.java
	core/src/main/java/org/carbondata/scan/expression/ExpressionResult.java
	core/src/main/java/org/carbondata/scan/expression/arithmetic/AddExpression.java
	core/src/main/java/org/carbondata/scan/expression/arithmetic/DivideExpression.java
	core/src/main/java/org/carbondata/scan/expression/arithmetic/MultiplyExpression.java
	core/src/main/java/org/carbondata/scan/expression/arithmetic/SubstractExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/EqualToExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanEqualToExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/GreaterThanExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/InExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/LessThanEqualToExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/LessThanExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/ListExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/NotEqualsExpression.java
	core/src/main/java/org/carbondata/scan/expression/conditional/NotInExpression.java
	core/src/main/java/org/carbondata/scan/expression/exception/FilterUnsupportedException.java
	core/src/main/java/org/carbondata/scan/expression/logical/AndExpression.java
	core/src/main/java/org/carbondata/scan/expression/logical/NotExpression.java
	core/src/main/java/org/carbondata/scan/expression/logical/OrExpression.java
	core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
	core/src/main/java/org/carbondata/scan/filter/executer/ColGroupFilterExecuterImpl.java
	core/src/main/java/org/carbondata/scan/filter/executer/RowLevelFilterExecuterImpl.java
	core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
	core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
	core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
	core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
	core/src/main/java/org/carbondata/scan/filter/executer/RowLevelRangeTypeExecuterFacory.java
	core/src/main/java/org/carbondata/scan/filter/resolver/AndFilterResolverImpl.java
	core/src/main/java/org/carbondata/scan/filter/resolver/FilterResolverIntf.java
	core/src/main/java/org/carbondata/scan/filter/resolver/LogicalFilterResolverImpl.java
	core/src/main/java/org/carbondata/scan/filter/resolver/RestructureFilterResolverImpl.java
	core/src/main/java/org/carbondata/scan/filter/resolver/RowLevelRangeFilterResolverImpl.java
	core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/CustomTypeDictionaryVisitor.java
	core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/DictionaryColumnVisitor.java
	core/src/main/java/org/carbondata/scan/filter/resolver/resolverinfo/visitor/NoDictionaryTypeVisitor.java
	core/src/main/java/org/carbondata/scan/util/DataTypeUtil.java
	integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase1.scala
	integration-testcases/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCase2.scala
	integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
	integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
	integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
	integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
	integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
	integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSQLDialect.scala
	integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonStrategies.scala
	integration/spark/src/main/scala/org/carbondata/spark/agg/CarbonAggregates.scala
	integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonScanRDD.scala
	integration/spark/src/test/scala/org/carbondata/spark/testsuite/allqueries/AllDataTypesTestCaseAggregate.scala
	processing/src/main/java/org/carbondata/processing/store/CarbonFactDataHandlerModel.java
	processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupDataHolder.java
	processing/src/main/java/org/carbondata/processing/store/colgroup/ColGroupMinMax.java
	processing/src/main/java/org/carbondata/processing/store/writer/AbstractFactDataWriter.java
	processing/src/test/java/org/carbondata/processing/store/colgroup/ColGroupMinMaxTest.java


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

Branch: refs/heads/master
Commit: 7f72218602926ca5f6629825df7b8f70ab1c6aa6
Parents: e9bbf75 d44c9e7
Author: ravipesala <ra...@gmail.com>
Authored: Thu Jun 30 17:41:47 2016 +0530
Committer: ravipesala <ra...@gmail.com>
Committed: Thu Jun 30 17:41:47 2016 +0530

----------------------------------------------------------------------
 common/pom.xml                                  |    8 +-
 .../common/logging/impl/StandardLogService.java |   15 +-
 conf/carbon.properties.template                 |   98 +
 .../common/ext/ColumnUniqueIdGenerator.java     |   41 +
 .../common/ext/DictionaryFactory.java           |  119 +
 .../org/carbondata/common/ext/PathFactory.java  |   48 +
 .../common/factory/CarbonCommonFactory.java     |   54 +
 .../dictionary/AbstractDictionaryCache.java     |   43 +-
 .../cache/dictionary/ColumnDictionaryInfo.java  |    5 +-
 .../cache/dictionary/DictionaryCacheLoader.java |    7 +-
 .../dictionary/DictionaryCacheLoaderImpl.java   |   28 +-
 .../DictionaryColumnUniqueIdentifier.java       |    9 +-
 .../dictionary/ForwardDictionaryCache.java      |   11 +-
 .../dictionary/ReverseDictionaryCache.java      |   11 +-
 .../core/carbon/ColumnIdentifier.java           |  109 +
 .../core/carbon/datastore/BlockIndexStore.java  |  184 +-
 .../carbon/datastore/SegmentTaskIndexStore.java |   21 +-
 .../datastore/block/SegmentProperties.java      |   94 +-
 .../carbon/datastore/block/TableBlockInfo.java  |   18 +-
 .../impl/ColumnGroupDimensionDataChunk.java     |    7 +-
 .../ThriftWrapperSchemaConverterImpl.java       |    4 +
 .../carbon/metadata/index/BlockIndexInfo.java   |   92 +
 .../metadata/schema/table/CarbonTable.java      |    2 +-
 .../schema/table/column/CarbonColumn.java       |   23 +
 .../schema/table/column/ColumnSchema.java       |   49 +
 .../core/carbon/path/CarbonTablePath.java       |  120 +-
 .../core/constants/CarbonCommonConstants.java   |  285 +-
 .../store/columnar/ColumnGroupModel.java        |   21 -
 .../store/filesystem/HDFSCarbonFile.java        |    3 +-
 .../store/filesystem/ViewFSCarbonFile.java      |    6 +-
 .../core/datastorage/util/StoreFactory.java     |   18 +-
 .../DirectDictionaryGenerator.java              |   13 +
 .../TimeStampDirectDictionaryGenerator.java     |   46 +-
 .../core/load/LoadMetadataDetails.java          |   43 +
 .../CarbonDictionaryMetadataReaderImpl.java     |   17 +-
 .../core/reader/CarbonDictionaryReaderImpl.java |   28 +-
 .../core/reader/CarbonIndexFileReader.java      |   95 +
 .../carbondata/core/reader/ThriftReader.java    |   47 +-
 .../CarbonDictionarySortIndexReaderImpl.java    |   21 +-
 .../core/service/ColumnUniqueIdService.java     |   34 +
 .../core/service/DictionaryService.java         |   92 +
 .../carbondata/core/service/PathService.java    |   38 +
 .../core/util/CarbonMetadataUtil.java           |  114 +-
 .../carbondata/core/util/CarbonProperties.java  |  401 +--
 .../org/carbondata/core/util/CarbonUtil.java    |  245 +-
 .../core/util/DataFileFooterConverter.java      |   65 +-
 .../core/writer/CarbonDictionaryWriterImpl.java |   45 +-
 .../core/writer/CarbonIndexFileWriter.java      |   64 +
 .../core/writer/HierarchyValueWriterForCSV.java |    2 +-
 .../CarbonDictionarySortIndexWriterImpl.java    |   21 +-
 .../scan/executor/infos/KeyStructureInfo.java   |   18 -
 .../scan/executor/util/QueryUtil.java           |  105 +-
 .../scan/expression/ColumnExpression.java       |    3 +-
 .../carbondata/scan/expression/Expression.java  |    4 +-
 .../scan/expression/ExpressionResult.java       |  104 +-
 .../expression/arithmetic/AddExpression.java    |    4 +-
 .../expression/arithmetic/DivideExpression.java |    4 +-
 .../arithmetic/MultiplyExpression.java          |    4 +-
 .../arithmetic/SubstractExpression.java         |    4 +-
 .../conditional/EqualToExpression.java          |    5 +-
 .../GreaterThanEqualToExpression.java           |    4 +-
 .../conditional/GreaterThanExpression.java      |    5 +-
 .../expression/conditional/InExpression.java    |    6 +-
 .../conditional/LessThanEqualToExpression.java  |    5 +-
 .../conditional/LessThanExpression.java         |    4 +-
 .../expression/conditional/ListExpression.java  |    7 +-
 .../conditional/NotEqualsExpression.java        |    4 +-
 .../expression/conditional/NotInExpression.java |    6 +-
 .../exception/FilterIllegalMemberException.java |   98 +
 .../exception/FilterUnsupportedException.java   |    1 -
 .../scan/expression/logical/AndExpression.java  |    6 +-
 .../scan/expression/logical/NotExpression.java  |    4 +-
 .../scan/expression/logical/OrExpression.java   |    4 +-
 .../scan/filter/FilterExpressionProcessor.java  |   36 +-
 .../org/carbondata/scan/filter/FilterUtil.java  |  163 +-
 .../executer/ColGroupFilterExecuterImpl.java    |  190 --
 .../ExcludeColGroupFilterExecuterImpl.java      |  137 +
 .../executer/ExcludeFilterExecuterImpl.java     |   49 +-
 .../IncludeColGroupFilterExecuterImpl.java      |  209 ++
 .../executer/IncludeFilterExecuterImpl.java     |    8 +-
 .../executer/RowLevelFilterExecuterImpl.java    |  116 +-
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  |  148 +-
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java |  139 +-
 ...velRangeLessThanEqualFilterExecuterImpl.java |  187 +-
 .../RowLevelRangeLessThanFiterExecuterImpl.java |  190 +-
 .../RowLevelRangeTypeExecuterFacory.java        |   13 +-
 .../filter/resolver/AndFilterResolverImpl.java  |    5 +-
 .../resolver/ConditionalFilterResolverImpl.java |   19 +-
 .../filter/resolver/FilterResolverIntf.java     |    5 +-
 .../resolver/LogicalFilterResolverImpl.java     |    3 +-
 .../resolver/RestructureFilterResolverImpl.java |    5 +-
 .../RowLevelRangeFilterResolverImpl.java        |   77 +-
 .../visitor/CustomTypeDictionaryVisitor.java    |   16 +-
 .../visitor/DictionaryColumnVisitor.java        |   11 +-
 .../visitor/NoDictionaryTypeVisitor.java        |   12 +-
 .../scan/result/iterator/RawResultIterator.java |    2 +
 .../org/carbondata/scan/util/DataTypeUtil.java  |    2 +-
 .../dictionary/AbstractDictionaryCacheTest.java |    7 +-
 .../dictionary/ForwardDictionaryCacheTest.java  |    4 +-
 .../dictionary/ReverseDictionaryCacheTest.java  |    4 +-
 .../datastore/SegmentTaskIndexStoreTest.java    |  143 -
 ...CarbonDictionarySortIndexReaderImplTest.java |    6 +-
 .../writer/CarbonDictionaryWriterImplTest.java  |    9 +-
 .../core/writer/CarbonFooterWriterTest.java     |   58 +-
 ...CarbonDictionarySortIndexWriterImplTest.java |   13 +-
 dev/molap.properties.template                   |   94 -
 dev/molap.properties_spark                      |   90 -
 .../examples/GenerateDictionaryExample.scala    |    4 +-
 format/src/main/thrift/carbondataindex.thrift   |   45 +
 format/src/main/thrift/schema.thrift            |   13 +-
 .../org/carbondata/hadoop/CarbonPathFilter.java |    4 +-
 .../AbstractDictionaryDecodedReadSupport.java   |    4 +-
 .../hadoop/test/util/StoreCreator.java          |   11 +-
 .../FACT_UNITED_DATA_INFO_sample_cube.csv       |   20 -
 .../FACT_UNITED_DATA_INFO_sample_table.csv      |   20 +
 .../src/test/resources/restructure_cube.csv     |  101 -
 .../src/test/resources/restructure_table.csv    |  101 +
 .../allqueries/AllDataTypesTestCase1.scala      |   82 +-
 .../allqueries/AllDataTypesTestCase2.scala      |  116 +-
 .../allqueries/AllDataTypesTestCase3.scala      |  211 +-
 .../allqueries/AllDataTypesTestCase4.scala      | 2576 +++++-------------
 .../allqueries/AllDataTypesTestCase5.scala      |  165 +-
 .../allqueries/AllDataTypesTestCase6.scala      | 2377 ++++------------
 .../spark/merger/RowResultMerger.java           |   19 +-
 .../carbondata/spark/load/CarbonLoadModel.java  |    2 +
 .../carbondata/spark/load/CarbonLoaderUtil.java |  189 +-
 .../spark/load/DeleteLoadFolders.java           |    5 +-
 .../spark/merger/CarbonDataMergerUtil.java      |  401 +--
 .../spark/sql/CarbonDatasourceRelation.scala    |   22 +-
 .../spark/sql/CarbonDictionaryDecoder.scala     |    7 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  374 +--
 .../execution/command/carbonTableSchema.scala   |  180 +-
 .../spark/sql/hive/CarbonMetastoreCatalog.scala |   85 +-
 .../spark/sql/hive/CarbonStrategies.scala       |    4 +-
 .../spark/CarbonColumnValidator.scala           |   36 +
 .../carbondata/spark/CarbonSparkFactory.scala   |   63 +
 .../spark/DictionaryDetailHelper.scala          |   66 +
 .../spark/rdd/CarbonDataLoadRDD.scala           |    4 +
 .../spark/rdd/CarbonDataRDDFactory.scala        |  240 +-
 .../spark/rdd/CarbonGlobalDictionaryRDD.scala   |    7 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   13 +-
 .../org/carbondata/spark/rdd/Compactor.scala    |   19 +-
 .../org/carbondata/spark/util/CommonUtil.scala  |   79 +-
 .../spark/util/GlobalDictionaryUtil.scala       |   85 +-
 .../spark/load/CarbonLoaderUtilTest.java        |    4 +-
 .../validation/FileFooterValidator.java         |    6 +-
 integration/spark/src/test/resources/data2.csv  |    6 +-
 .../src/test/resources/data2_DiffTimeFormat.csv |    4 +
 .../resources/datawithcomplexspecialchar.csv    |  151 +
 .../test/resources/datawithescapecharacter.csv  |   22 +
 .../src/test/resources/datawithnullmeasure.csv  |    5 +
 .../test/resources/datawithspecialcharacter.csv |   38 +
 .../spark/src/test/resources/filter/emp2.csv    |    9 +
 .../src/test/resources/filter/emp2allnull.csv   |    9 +
 .../src/test/resources/filter/emp2nonull.csv    |    8 +
 .../test/resources/nullandnonparsableValue.csv  |    3 +
 .../apache/spark/sql/TestCarbonSqlParser.scala  |   37 +-
 .../dataload/TestLoadDataWithBlankLine.scala    |   18 +-
 .../TestLoadDataWithEmptyArrayColumns.scala     |   12 +-
 .../TestLoadDataWithMaxMinInteger.scala         |    6 +-
 .../dataload/TestLoadDataWithNullMeasures.scala |   15 +-
 .../AllDataTypesTestCaseAggregate.scala         |   44 +-
 .../aggquery/IntegerDataTypeTestCase.scala      |    7 +-
 .../AllDataTypesTestCaseAggregate.scala         |    8 +-
 .../testsuite/bigdecimal/TestBigDecimal.scala   |   10 +
 .../bigdecimal/TestNullAndEmptyFields.scala     |  110 +
 .../createtable/TestCreateTableSyntax.scala     |   32 +-
 .../DataCompactionCardinalityBoundryTest.scala  |   12 +-
 .../datacompaction/DataCompactionLockTest.scala |  111 +
 .../datacompaction/DataCompactionTest.scala     |   12 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |  108 +-
 ...ataWithMalformedCarbonCommandException.scala |    4 +-
 .../dataretention/DataRetentionTestCase.scala   |   37 +
 .../detailquery/AllDataTypesTestCase.scala      |    6 +-
 .../ColumnGroupDataTypesTestCase.scala          |   51 +-
 .../ColumnPropertyValidationTestCase.scala      |   28 +
 .../HighCardinalityDataTypesTestCase.scala      |   66 +-
 .../detailquery/IntegerDataTypeTestCase.scala   |    6 +-
 ...estampDataTypeDirectDictionaryTestCase.scala |   12 +-
 ...TypeDirectDictionaryWithNoDictTestCase.scala |    4 +-
 .../filterexpr/AllDataTypesTestCaseFilter.scala |    6 +-
 .../filterexpr/FilterProcessorTestCase.scala    |  131 +-
 .../GrtLtFilterProcessorTestCase.scala          |  176 ++
 .../filterexpr/IntegerDataTypeTestCase.scala    |    6 +-
 .../NullMeasureValueTestCaseFilter.scala        |   39 +
 .../HadoopFSRelationTestCase.scala              |   16 +-
 .../joinquery/AllDataTypesTestCaseJoin.scala    |    6 +-
 .../joinquery/IntegerDataTypeTestCase.scala     |    6 +-
 .../NullMeasureValueTestCaseAggregate.scala     |    2 +-
 .../sortexpr/AllDataTypesTestCaseSort.scala     |    6 +-
 .../sortexpr/IntegerDataTypeTestCase.scala      |   13 +-
 .../util/GlobalDictionaryUtilTestCase.scala     |   51 +-
 .../carbonaggregatesurrogategenerator/icon.png  |  Bin 9058 -> 0 bytes
 .../plugin.xml                                  |    1 -
 .../steps/carbonautoagggraphgenerator/icon.png  |  Bin 9058 -> 0 bytes
 .../carbonautoagggraphgenerator/plugin.xml      |    1 -
 .../steps/carbonautoaggslicemerger/icon.png     |  Bin 9058 -> 0 bytes
 .../steps/carbonautoaggslicemerger/plugin.xml   |    1 -
 .../plugins/steps/carboncsvbasedseqgen/icon.png |  Bin 24093 -> 0 bytes
 .../steps/carboncsvbasedseqgen/plugin.xml       |    1 -
 .../plugins/steps/carboncsvreader/icon.png      |  Bin 24093 -> 0 bytes
 .../plugins/steps/carboncsvreader/plugin.xml    |    1 -
 .../plugins/steps/carboncsvreaderstrep/icon.png |  Bin 9058 -> 0 bytes
 .../steps/carboncsvreaderstrep/plugin.xml       |    1 -
 .../steps/carbondatawriter/Binary-icon.png      |  Bin 14573 -> 0 bytes
 .../plugins/steps/carbondatawriter/plugin.xml   |    1 -
 .../plugins/steps/carbonfactreader/icon.png     |  Bin 9058 -> 0 bytes
 .../plugins/steps/carbonfactreader/plugin.xml   |    1 -
 .../plugins/steps/carbongroupby/icon.png        |  Bin 9058 -> 0 bytes
 .../plugins/steps/carbongroupby/plugin.xml      |    1 -
 .../steps/carboninmemoryfactreader/icon.png     |  Bin 9058 -> 0 bytes
 .../steps/carboninmemoryfactreader/plugin.xml   |    1 -
 .../.kettle/plugins/steps/carbonseqgen/icon.png |  Bin 24093 -> 0 bytes
 .../plugins/steps/carbonseqgen/plugin.xml       |    1 -
 .../plugins/steps/carbonslicemerger/icon.png    |  Bin 9058 -> 0 bytes
 .../plugins/steps/carbonslicemerger/plugin.xml  |    1 -
 .../steps/carbonsortkeyandgroupby/icon.png      |  Bin 9058 -> 0 bytes
 .../steps/carbonsortkeyandgroupby/plugin.xml    |    1 -
 .../.kettle/plugins/steps/mdkeygenstep/icon.png |  Bin 9058 -> 0 bytes
 .../plugins/steps/mdkeygenstep/plugin.xml       |    1 -
 .../.kettle/plugins/steps/sortkeystep/icon.png  |  Bin 9058 -> 0 bytes
 .../plugins/steps/sortkeystep/plugin.xml        |    1 -
 .../org/carbondata/lcm/locks/LocalFileLock.java |    4 +-
 .../org/carbondata/lcm/locks/LockUsage.java     |    3 +-
 .../lcm/status/SegmentStatusManager.java        |  111 +-
 .../api/dataloader/DataLoadModel.java           |    9 +
 .../processing/csvload/DataGraphExecuter.java   |    8 +
 .../csvreaderstep/BlockDataHandler.java         |   83 +-
 .../processing/csvreaderstep/CsvInput.java      |    4 +-
 .../processing/csvreaderstep/CsvInputData.java  |    2 +
 .../processing/csvreaderstep/CsvInputMeta.java  |   21 +-
 .../dataprocessor/DataProcessTaskStatus.java    |   10 +
 .../dataprocessor/IDataProcessStatus.java       |    2 +
 .../graphgenerator/GraphGenerator.java          |    6 +
 .../configuration/GraphConfigurationInfo.java   |   10 +
 .../processing/mdkeygen/MDKeyGenStep.java       |   40 +-
 .../merger/step/CarbonSliceMergerStep.java      |    3 +-
 .../processing/schema/metadata/ColumnsInfo.java |   10 +
 .../store/CarbonFactDataHandlerColumnar.java    |  111 +-
 .../store/CarbonFactDataHandlerModel.java       |   63 +-
 .../store/SingleThreadFinalSortFilesMerger.java |    6 +-
 .../store/colgroup/ColGroupDataHolder.java      |    4 +-
 .../store/colgroup/ColGroupMinMax.java          |   30 +-
 .../store/writer/AbstractFactDataWriter.java    |  147 +-
 ...actDataWriterImplForIntIndexAndAggBlock.java |    6 +-
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      |   36 +
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |   19 +-
 .../FileStoreSurrogateKeyGenForCSV.java         |   12 +-
 .../processing/util/CarbonSchemaParser.java     |   44 +
 .../store/colgroup/ColGroupMinMaxTest.java      |  385 +--
 250 files changed, 8562 insertions(+), 7434 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/cache/dictionary/ColumnDictionaryInfo.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
index c177feb,f6f369a..bfa5cdc
--- a/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
+++ b/core/src/main/java/org/carbondata/core/carbon/datastore/block/SegmentProperties.java
@@@ -32,6 -34,6 +34,7 @@@ import org.carbondata.core.carbon.metad
  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.ColumnGroupModel;
  import org.carbondata.core.keygenerator.KeyGenerator;
  import org.carbondata.core.keygenerator.columnar.ColumnarSplitter;
  import org.carbondata.core.keygenerator.columnar.impl.MultiDimKeyVarLengthVariableSplitGenerator;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/carbon/datastore/chunk/impl/ColumnGroupDimensionDataChunk.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/constants/CarbonCommonConstants.java
----------------------------------------------------------------------

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnGroupModel.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/core/datastorage/store/columnar/ColumnGroupModel.java
index 532c6e9,0000000..26b2519
mode 100644,000000..100644
--- 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
@@@ -1,137 -1,0 +1,116 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
 + * KIND, either express or implied.  See the License for the
 + * specific language governing permissions and limitations
 + * under the License.
 + */
 +package org.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/7f722186/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
index 8fc5aac,63f8157..e8efa21
--- a/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
+++ b/core/src/main/java/org/carbondata/core/datastorage/util/StoreFactory.java
@@@ -19,35 -19,38 +19,19 @@@
  
  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.util.CarbonProperties;
 -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;
  
  public final class StoreFactory {
    /**
-    * Double array data store.
 -   * key type.
--   */
-   private static final String COMPRESSED_DOUBLE_ARRAY = "COMPRESSED_DOUBLE_ARRAY";
 -  private static StoreType keyType;
--  /**
     * value type.
     */
    private static StoreType valueType;
  
    static {
-     String valuetype = CarbonProperties.getInstance()
-         .getProperty(CarbonCommonConstants.VALUESTORE_TYPE,
-             CarbonCommonConstants.VALUESTORE_TYPE_DEFAULT_VAL);
- 
-     // set value type
-     if (COMPRESSED_DOUBLE_ARRAY.equals(valuetype)) {
-       valueType = StoreType.COMPRESSED_DOUBLE_ARRAY;
-     } else {
-       valueType = StoreType.HEAVY_VALUE_COMPRESSION;
-     }
 -    keyType = StoreType.COMPRESSED_SINGLE_ARRAY;
+     valueType = StoreType.HEAVY_VALUE_COMPRESSION;
    }
  
    private StoreFactory() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/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 771e5f5,97baf3d..0320621
--- 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
@@@ -39,20 -39,7 +39,14 @@@ import static org.carbondata.core.keyge
   */
  public class TimeStampDirectDictionaryGenerator implements DirectDictionaryGenerator {
  
 +  private TimeStampDirectDictionaryGenerator() {
 +
 +  }
 +
 +  public static TimeStampDirectDictionaryGenerator instance =
 +      new TimeStampDirectDictionaryGenerator();
 +
    /**
-    * Logger instance
-    */
-   private static final LogService LOGGER =
-       LogServiceFactory.getLogService(TimeStampDirectDictionaryGenerator.class.getName());
- 
-   /**
     * The value of 1 unit of the SECOND, MINUTE, HOUR, or DAY in millis.
     */
    public static final long granularityFactor;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 31d41b3,b19f8d3..2368139
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@@ -47,9 -40,14 +40,13 @@@ import org.carbondata.core.carbon.metad
  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.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.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;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/7f722186/core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
index 6641c18,0000000..52a139c
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
+++ b/core/src/main/java/org/carbondata/core/util/DataFileFooterConverter.java
@@@ -1,418 -1,0 +1,463 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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.datastore.block.TableBlockInfo;
 +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.reader.CarbonIndexFileReader;
++import org.carbondata.format.BlockIndex;
 +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 {
++public class DataFileFooterConverter {
 +
 +  private static final LogService LOGGER =
 +      LogServiceFactory.getLogService(DataFileFooterConverter.class.getName());
 +
 +  /**
++   * Below method will be used to get the index info from index file
++   *
++   * @param filePath           file path of the index file
++   * @param tableBlockInfoList table block index
++   * @return list of index info
++   * @throws IOException problem while reading the index file
++   */
++  public List<DataFileFooter> getIndexInfo(String filePath, List<TableBlockInfo> tableBlockInfoList)
++      throws IOException {
++    CarbonIndexFileReader indexReader = new CarbonIndexFileReader();
++    // open the reader
++    indexReader.openThriftReader(filePath);
++    // get the index header
++    org.carbondata.format.IndexHeader readIndexHeader = indexReader.readIndexHeader();
++    List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
++    List<org.carbondata.format.ColumnSchema> table_columns = readIndexHeader.getTable_columns();
++    for (int i = 0; i < table_columns.size(); i++) {
++      columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
++    }
++    // get the segment info
++    SegmentInfo segmentInfo = getSegmentInfo(readIndexHeader.getSegment_info());
++    BlockletIndex blockletIndex = null;
++    int counter = 0;
++    DataFileFooter dataFileFooter = null;
++    List<DataFileFooter> dataFileFooters = new ArrayList<DataFileFooter>();
++    // read the block info from file
++    while (indexReader.hasNext()) {
++      BlockIndex readBlockIndexInfo = indexReader.readBlockIndexInfo();
++      blockletIndex = getBlockletIndex(readBlockIndexInfo.getBlock_index());
++      dataFileFooter = new DataFileFooter();
++      dataFileFooter.setBlockletIndex(blockletIndex);
++      dataFileFooter.setColumnInTable(columnSchemaList);
++      dataFileFooter.setNumberOfRows(readBlockIndexInfo.getNum_rows());
++      dataFileFooter.setTableBlockInfo(tableBlockInfoList.get(counter++));
++      dataFileFooter.setSegmentInfo(segmentInfo);
++      dataFileFooters.add(dataFileFooter);
++    }
++    return dataFileFooters;
++  }
++
++  /**
 +   * 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));
++    if (null != column_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 SHORT:
 +        return DataType.SHORT;
 +      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/7f722186/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
----------------------------------------------------------------------
diff --cc core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
index 51874b4,0000000..e19a3e9
mode 100644,000000..100644
--- a/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
+++ b/core/src/main/java/org/carbondata/scan/executor/infos/KeyStructureInfo.java
@@@ -1,125 -1,0 +1,107 @@@
 +/*
 + * Licensed to the Apache Software Foundation (ASF) under one
 + * or more contributor license agreements.  See the NOTICE file
 + * distributed with this work for additional information
 + * regarding copyright ownership.  The ASF licenses this file
 + * to you under the Apache License, Version 2.0 (the
 + * "License"); you may not use this file except in compliance
 + * with the License.  You may obtain a copy of the License at
 + *
 + *    http://www.apache.org/licenses/LICENSE-2.0
 + *
 + * Unless required by applicable law or agreed to in writing,
 + * software distributed under the License is distributed on an
 + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS 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;
-   }
 +}


[28/50] [abbrv] incubator-carbondata git commit: [CARBONDATA-13] Direct surrogate key range filters not able to prune blocks (#721)

Posted by ch...@apache.org.
[CARBONDATA-13] Direct surrogate key range filters not able to prune blocks (#721)

* When Range filters(>,<,>=,<=) is applied Direct Surrogate Column block and blocklet pruning is supported
* Binary search inside block let for range filters.

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

Branch: refs/heads/master
Commit: 462eb393baf99d3b0a7f49869f5a3d95f0e2fd9e
Parents: 9d846e4
Author: sujith71955 <su...@gmail.com>
Authored: Mon Jun 27 22:34:51 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Mon Jun 27 22:34:51 2016 +0530

----------------------------------------------------------------------
 .../org/carbondata/core/util/CarbonUtil.java    |  45 +++++-
 .../executer/ExcludeFilterExecuterImpl.java     |   4 +-
 .../executer/IncludeFilterExecuterImpl.java     |   4 +-
 .../RowLevelRangeGrtThanFiterExecuterImpl.java  | 147 ++++++++++++++++++-
 ...elRangeGrtrThanEquaToFilterExecuterImpl.java | 139 +++++++++++++++++-
 ...velRangeLessThanEqualFilterExecuterImpl.java | 137 ++++++++++++++++-
 .../RowLevelRangeLessThanFiterExecuterImpl.java | 141 +++++++++++++++++-
 .../RowLevelRangeTypeExecuterFacory.java        |  13 +-
 .../filter/resolver/AndFilterResolverImpl.java  |   5 +-
 .../resolver/ConditionalFilterResolverImpl.java |  19 +--
 .../filter/resolver/FilterResolverIntf.java     |   5 +-
 .../resolver/LogicalFilterResolverImpl.java     |   3 +-
 .../RowLevelRangeFilterResolverImpl.java        |  55 ++++++-
 .../filters/FilterExpressionProcessor.java      |  15 +-
 .../filters/measurefilter/util/FilterUtil.java  |   3 +-
 integration/spark/src/test/resources/data2.csv  |   6 +-
 .../HighCardinalityDataTypesTestCase.scala      |   4 +
 .../filterexpr/FilterProcessorTestCase.scala    |  78 +++++++---
 18 files changed, 745 insertions(+), 78 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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 886e136..12cbb6b 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -888,7 +888,50 @@ public final class CarbonUtil {
         return currentIndex;
       }
     }
-    return -1;
+    return -(low + 1);
+  }
+
+  /**
+   * Method will identify the value which is lesser than the pivot element
+   * on which range filter is been applied.
+   *
+   * @param currentIndex
+   * @param dimColumnDataChunk
+   * @param compareValue
+   * @return index value
+   */
+  public static int nextLesserValueToTarget(int currentIndex,
+      FixedLengthDimensionDataChunk dimColumnDataChunk, byte[] compareValue) {
+    while (currentIndex - 1 >= 0 && ByteUtil.UnsafeComparer.INSTANCE
+        .compareTo(dimColumnDataChunk.getCompleteDataChunk(),
+            (currentIndex - 1) * compareValue.length, compareValue.length, compareValue, 0,
+            compareValue.length) >= 0) {
+      --currentIndex;
+    }
+
+    return --currentIndex;
+  }
+
+  /**
+   * Method will identify the value which is greater than the pivot element
+   * on which range filter is been applied.
+   *
+   * @param currentIndex
+   * @param dimColumnDataChunk
+   * @param compareValue
+   * @param numerOfRows
+   * @return index value
+   */
+  public static int nextGreaterValueToTarget(int currentIndex,
+      FixedLengthDimensionDataChunk dimColumnDataChunk, byte[] compareValue, int numerOfRows) {
+    while (currentIndex + 1 < numerOfRows && ByteUtil.UnsafeComparer.INSTANCE
+        .compareTo(dimColumnDataChunk.getCompleteDataChunk(),
+            (currentIndex + 1) * compareValue.length, compareValue.length, compareValue, 0,
+            compareValue.length) <= 0) {
+      ++currentIndex;
+    }
+
+    return ++currentIndex;
   }
 
   public static int[] getUnCompressColumnIndex(int totalLength, byte[] columnIndexData,

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 9de8396..f2c6de3 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/ExcludeFilterExecuterImpl.java
@@ -140,7 +140,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       startKey = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
               filterValues[i]);
-      if (startKey == -1) {
+      if (startKey < 0) {
         continue;
       }
       bitSet.flip(columnIndex[startKey]);
@@ -175,7 +175,7 @@ public class ExcludeFilterExecuterImpl implements FilterExecuter {
       startKey = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimColumnDataChunk, startIndex, numerOfRows - 1,
               filterValues[k]);
-      if (startKey == -1) {
+      if (startKey < 0) {
         continue;
       }
       bitSet.flip(startKey);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index a15733c..5123ce7 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/IncludeFilterExecuterImpl.java
@@ -131,7 +131,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
       start = CarbonUtil
           .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
               filterValues[i]);
-      if (start == -1) {
+      if (start < 0) {
         continue;
       }
       bitSet.set(columnIndex[start]);
@@ -168,7 +168,7 @@ public class IncludeFilterExecuterImpl implements FilterExecuter {
         start = CarbonUtil.getFirstIndexUsingBinarySearch(
             (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
             filterValues[k]);
-        if (start == -1) {
+        if (start < 0) {
           continue;
         }
         bitSet.set(start);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 88290d9..571f046 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtThanFiterExecuterImpl.java
@@ -22,20 +22,30 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+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.metadata.encoder.Encoding;
 import org.carbondata.core.util.ByteUtil;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.carbon.processor.BlocksChunkHolder;
 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;
 
 public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
   private byte[][] filterRangeValues;
+  private SegmentProperties segmentProperties;
 
   public RowLevelRangeGrtThanFiterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
+      SegmentProperties segmentProperties) {
     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
     this.filterRangeValues = filterRangeValues;
+    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
@@ -49,7 +59,6 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
       // 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) {
@@ -63,4 +72,138 @@ public class RowLevelRangeGrtThanFiterExecuterImpl extends RowLevelFilterExecute
     return bitSet;
 
   }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
+      throws FilterUnsupportedException {
+    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return super.applyFilter(blockChunkHolder);
+    }
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
+    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
+      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
+          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    }
+    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
+        blockChunkHolder.getDataBlock().nodeSize());
+  }
+
+  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
+        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      return setFilterdIndexToBitSetWithColumnIndex(
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+    }
+    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all members
+   * will be considered for applying range filters. this method will be called if the
+   * column is not supported by default so column index mapping  will be present for
+   * accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @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 = this.filterRangeValues;
+    for (int i = 0; i < filterValues.length; i++) {
+      start = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+              filterValues[i]);
+      start = CarbonUtil
+          .nextGreaterValueToTarget(start, (FixedLengthDimensionDataChunk) dimensionColumnDataChunk,
+              filterValues[i], numerOfRows);
+      // Logic will handle the case where the range filter member is not present in block
+      // in this case the binary search will return the index from where the bit sets will be
+      // set inorder to apply filters. this is greater than filter so the range will be taken
+      // from the next element which is greater than filter member.
+      if (start < 0) {
+        start = -(start + 1);
+        if (start == numerOfRows) {
+          start = start - 1;
+        }
+        // Method will compare the tentative index value after binary search, this tentative
+        // index needs to be compared by the filter member if its > filter then from that
+        // index the bitset will be considered for filtering process.
+        if (ByteUtil
+            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
+            > 0) {
+          start = start + 1;
+        }
+      }
+
+      last = start;
+      for (int j = start; j < numerOfRows; j++) {
+        bitSet.set(columnIndex[j]);
+        last++;
+      }
+      startIndex = last;
+      if (startIndex >= numerOfRows) {
+        break;
+      }
+    }
+
+    return bitSet;
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all
+   * members will be considered for applying range filters. this method will
+   * be called if the column is sorted default so column index
+   * mapping will be present for accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @return BitSet.
+   */
+  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      int start = 0;
+      int last = 0;
+      int startIndex = 0;
+      byte[][] filterValues = this.filterRangeValues;
+      for (int k = 0; k < filterValues.length; k++) {
+        start = CarbonUtil.getFirstIndexUsingBinarySearch(
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            filterValues[k]);
+        start = CarbonUtil.nextGreaterValueToTarget(start,
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[k], numerOfRows);
+        if (start < 0) {
+          start = -(start + 1);
+          if (start == numerOfRows) {
+            start = start - 1;
+          }
+          // Method will compare the tentative index value after binary search, this tentative
+          // index needs to be compared by the filter member if its > filter then from that
+          // index the bitset will be considered for filtering process.
+          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) > 0) {
+            start = start + 1;
+          }
+        }
+        last = start;
+        for (int j = start; j < numerOfRows; j++) {
+          bitSet.set(j);
+          last++;
+        }
+        startIndex = last;
+        if (startIndex >= numerOfRows) {
+          break;
+        }
+      }
+    }
+    return bitSet;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 4d7d962..e390b8d 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeGrtrThanEquaToFilterExecuterImpl.java
@@ -22,21 +22,31 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+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.metadata.encoder.Encoding;
 import org.carbondata.core.util.ByteUtil;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.carbon.processor.BlocksChunkHolder;
 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;
 
 public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilterExecuterImpl {
 
-  private byte[][] filterRangeValues;
+  protected byte[][] filterRangeValues;
+  private SegmentProperties segmentProperties;
 
   public RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
+      SegmentProperties segmentProperties) {
     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
     this.filterRangeValues = filterRangeValues;
+    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
@@ -63,4 +73,129 @@ public class RowLevelRangeGrtrThanEquaToFilterExecuterImpl extends RowLevelFilte
     return bitSet;
 
   }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
+      throws FilterUnsupportedException {
+    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return super.applyFilter(blockChunkHolder);
+    }
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
+    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
+      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
+          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    }
+    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
+        blockChunkHolder.getDataBlock().nodeSize());
+  }
+
+  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
+        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      return setFilterdIndexToBitSetWithColumnIndex(
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+    }
+    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all members
+   * will be considered for applying range filters. this method will be called if the
+   * column is not supported by default so column index mapping  will be present for
+   * accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @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 = this.filterRangeValues;
+    for (int i = 0; i < filterValues.length; i++) {
+      start = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+              filterValues[i]);
+      if (start < 0) {
+        start = -(start + 1);
+        if (start == numerOfRows) {
+          start = start - 1;
+        }
+        // Method will compare the tentative index value after binary search, this tentative
+        // index needs to be compared by the filter member if its >= filter then from that
+        // index the bitset will be considered for filtering process.
+        if (ByteUtil
+            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
+            >= 0) {
+          start = start + 1;
+        }
+      }
+      last = start;
+      for (int j = start; j < numerOfRows; j++) {
+
+        bitSet.set(columnIndex[j]);
+        last++;
+      }
+      startIndex = last;
+      if (startIndex >= numerOfRows) {
+        break;
+      }
+    }
+    return bitSet;
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all
+   * members will be considered for applying range filters. this method will
+   * be called if the column is sorted default so column index
+   * mapping will be present for accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @return BitSet.
+   */
+  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      int start = 0;
+      int last = 0;
+      int startIndex = 0;
+      byte[][] filterValues = this.filterRangeValues;
+      for (int k = 0; k < filterValues.length; k++) {
+        start = CarbonUtil.getFirstIndexUsingBinarySearch(
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            filterValues[k]);
+        if (start < 0) {
+          start = -(start + 1);
+          if (start == numerOfRows) {
+            start = start - 1;
+          }
+          // Method will compare the tentative index value after binary search, this tentative
+          // index needs to be compared by the filter member if its >= filter then from that
+          // index the bitset will be considered for filtering process.
+          if (ByteUtil.compare(filterValues[k],dimensionColumnDataChunk.getChunkData(start))
+              >= 0) {
+            start = start + 1;
+          }
+        }
+
+        last = start;
+        for (int j = start; j < numerOfRows; j++) {
+          bitSet.set(j);
+          last++;
+        }
+        startIndex = last;
+        if (startIndex >= numerOfRows) {
+          break;
+        }
+      }
+    }
+    return bitSet;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index e5f10ef..1f33c8c 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanEqualFilterExecuterImpl.java
@@ -22,20 +22,30 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+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.metadata.encoder.Encoding;
 import org.carbondata.core.util.ByteUtil;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.carbon.processor.BlocksChunkHolder;
 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;
 
 public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilterExecuterImpl {
-  private byte[][] filterRangeValues;
+  protected byte[][] filterRangeValues;
+  private SegmentProperties segmentProperties;
 
   public RowLevelRangeLessThanEqualFilterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
+      SegmentProperties segmentProperties) {
     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
     this.filterRangeValues = filterRangeValues;
+    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
@@ -63,4 +73,127 @@ public class RowLevelRangeLessThanEqualFilterExecuterImpl extends RowLevelFilter
 
   }
 
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
+      throws FilterUnsupportedException {
+    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return super.applyFilter(blockChunkHolder);
+    }
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
+    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
+      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
+          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    }
+    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
+        blockChunkHolder.getDataBlock().nodeSize());
+  }
+
+  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
+        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      return setFilterdIndexToBitSetWithColumnIndex(
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+    }
+    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all members
+   * will be considered for applying range filters. this method will be called if the
+   * column is not supported by default so column index mapping  will be present for
+   * accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @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 = this.filterRangeValues;
+    for (int i = 0; i < filterValues.length; i++) {
+      start = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+              filterValues[i]);
+      if (start < 0) {
+        start = -(start + 1);
+        if (start == numerOfRows) {
+          start = start - 1;
+        }
+        // Method will compare the tentative index value after binary search, this tentative
+        // index needs to be compared by the filter member if its >= filter then from that
+        // index the bitset will be considered for filtering process.
+        if (ByteUtil
+            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
+            <= 0) {
+          start = start - 1;
+        }
+      }
+      last = start;
+      for (int j = start; j >= 0; j--) {
+        bitSet.set(columnIndex[j]);
+        last--;
+      }
+      startIndex = last;
+      if (startIndex <= 0) {
+        break;
+      }
+    }
+    return bitSet;
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all
+   * members will be considered for applying range filters. this method will
+   * be called if the column is sorted default so column index
+   * mapping will be present for accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @return BitSet.
+   */
+  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      int start = 0;
+      int last = 0;
+      int startIndex = 0;
+      byte[][] filterValues = this.filterRangeValues;
+      for (int k = 0; k < filterValues.length; k++) {
+        start = CarbonUtil.getFirstIndexUsingBinarySearch(
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            filterValues[k]);
+        if (start < 0) {
+          start = -(start + 1);
+          if (start == numerOfRows) {
+            start = start - 1;
+          }
+          // Method will compare the tentative index value after binary search, this tentative
+          // index needs to be compared by the filter member if its <= filter then from that
+          // index the bitset will be considered for filtering process.
+          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start))
+              <= 0) {
+            start = start - 1;
+          }
+        }
+        last = start;
+        for (int j = start; j >= 0; j--) {
+          bitSet.set(j);
+          last--;
+        }
+        startIndex = last;
+        if (startIndex <= 0) {
+          break;
+        }
+      }
+    }
+    return bitSet;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 56f7393..9bb0420 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeLessThanFiterExecuterImpl.java
@@ -22,20 +22,30 @@ import java.util.BitSet;
 import java.util.List;
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
+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.metadata.encoder.Encoding;
 import org.carbondata.core.util.ByteUtil;
+import org.carbondata.core.util.CarbonUtil;
+import org.carbondata.query.carbon.processor.BlocksChunkHolder;
 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;
 
 public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecuterImpl {
   private byte[][] filterRangeValues;
+  private SegmentProperties segmentProperties;
 
   public RowLevelRangeLessThanFiterExecuterImpl(
       List<DimColumnResolvedFilterInfo> dimColEvaluatorInfoList,
       List<MeasureColumnResolvedFilterInfo> msrColEvalutorInfoList, Expression exp,
-      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues) {
+      AbsoluteTableIdentifier tableIdentifier, byte[][] filterRangeValues,
+      SegmentProperties segmentProperties) {
     super(dimColEvaluatorInfoList, msrColEvalutorInfoList, exp, tableIdentifier);
     this.filterRangeValues = filterRangeValues;
+    this.segmentProperties = segmentProperties;
   }
 
   @Override public BitSet isScanRequired(byte[][] blockMaxValue, byte[][] blockMinValue) {
@@ -62,4 +72,133 @@ public class RowLevelRangeLessThanFiterExecuterImpl extends RowLevelFilterExecut
     return bitSet;
 
   }
+
+  @Override public BitSet applyFilter(BlocksChunkHolder blockChunkHolder)
+      throws FilterUnsupportedException {
+    if (!dimColEvaluatorInfoList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      return super.applyFilter(blockChunkHolder);
+    }
+    int blockIndex = segmentProperties.getDimensionOrdinalToBlockMapping()
+        .get(dimColEvaluatorInfoList.get(0).getColumnIndex());
+    if (null == blockChunkHolder.getDimensionDataChunk()[blockIndex]) {
+      blockChunkHolder.getDimensionDataChunk()[blockIndex] = blockChunkHolder.getDataBlock()
+          .getDimensionChunk(blockChunkHolder.getFileReader(), blockIndex);
+    }
+    return getFilteredIndexes(blockChunkHolder.getDimensionDataChunk()[blockIndex],
+        blockChunkHolder.getDataBlock().nodeSize());
+  }
+
+  private BitSet getFilteredIndexes(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    if (null != dimensionColumnDataChunk.getAttributes().getInvertedIndexes()
+        && dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      return setFilterdIndexToBitSetWithColumnIndex(
+          (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, numerOfRows);
+    }
+    return setFilterdIndexToBitSet(dimensionColumnDataChunk, numerOfRows);
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all members
+   * will be considered for applying range filters. this method will be called if the
+   * column is not supported by default so column index mapping  will be present for
+   * accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @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 = this.filterRangeValues;
+    for (int i = 0; i < filterValues.length; i++) {
+      start = CarbonUtil
+          .getFirstIndexUsingBinarySearch(dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+              filterValues[i]);
+      // Logic will handle the case where the range filter member is not present in block
+      // in this case the binary search will return the index from where the bit sets will be
+      // set inorder to apply filters. this is Lesser than filter so the range will be taken
+      // from the prev element which is Lesser than filter member.
+      start = CarbonUtil.nextLesserValueToTarget(start, dimensionColumnDataChunk, filterValues[i]);
+      if (start < 0) {
+        start = -(start + 1);
+        if (start == numerOfRows) {
+          start = start - 1;
+        }
+        // Method will compare the tentative index value after binary search, this tentative
+        // index needs to be compared by the filter member if its < filter then from that
+        // index the bitset will be considered for filtering process.
+        if (ByteUtil
+            .compare(filterValues[i], dimensionColumnDataChunk.getChunkData(columnIndex[start]))
+            < 0) {
+          start = start - 1;
+        }
+      }
+      last = start;
+      for (int j = start; j >= 0; j--) {
+        bitSet.set(columnIndex[j]);
+        last--;
+      }
+      startIndex = last;
+      if (startIndex >= 0) {
+        break;
+      }
+    }
+    return bitSet;
+  }
+
+  /**
+   * Method will scan the block and finds the range start index from which all
+   * members will be considered for applying range filters. this method will
+   * be called if the column is sorted default so column index
+   * mapping will be present for accesing the members from the block.
+   *
+   * @param dimensionColumnDataChunk
+   * @param numerOfRows
+   * @return BitSet.
+   */
+  private BitSet setFilterdIndexToBitSet(DimensionColumnDataChunk dimensionColumnDataChunk,
+      int numerOfRows) {
+    BitSet bitSet = new BitSet(numerOfRows);
+    if (dimensionColumnDataChunk instanceof FixedLengthDimensionDataChunk) {
+      int start = 0;
+      int last = 0;
+      int startIndex = 0;
+      byte[][] filterValues = this.filterRangeValues;
+      for (int k = 0; k < filterValues.length; k++) {
+        start = CarbonUtil.getFirstIndexUsingBinarySearch(
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, startIndex, numerOfRows - 1,
+            filterValues[k]);
+        start = CarbonUtil.nextLesserValueToTarget(start,
+            (FixedLengthDimensionDataChunk) dimensionColumnDataChunk, filterValues[k]);
+        if (start < 0) {
+          start = -(start + 1);
+          if (start == numerOfRows) {
+            start = start - 1;
+          }
+          // Method will compare the tentative index value after binary search, this tentative
+          // index needs to be compared by the filter member if its < filter then from that
+          // index the bitset will be considered for filtering process.
+          if (ByteUtil.compare(filterValues[k], dimensionColumnDataChunk.getChunkData(start)) < 0) {
+            start = start - 1;
+          }
+        }
+        last = start;
+        for (int j = start; j >= 0; j--) {
+          bitSet.set(j);
+          last--;
+        }
+        startIndex = last;
+        if (startIndex <= 0) {
+          break;
+        }
+      }
+    }
+    return bitSet;
+  }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 5a5f2d2..25e8a3f 100644
--- a/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeTypeExecuterFacory.java
+++ b/core/src/main/java/org/carbondata/query/filter/executer/RowLevelRangeTypeExecuterFacory.java
@@ -18,6 +18,7 @@
  */
 package org.carbondata.query.filter.executer;
 
+import org.carbondata.core.carbon.datastore.block.SegmentProperties;
 import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
 import org.carbondata.query.filter.resolver.FilterResolverIntf;
 import org.carbondata.query.filter.resolver.RowLevelRangeFilterResolverImpl;
@@ -33,11 +34,13 @@ public class RowLevelRangeTypeExecuterFacory {
    * filter tree resolver type.
    *
    * @param filterExpressionResolverTree
+   * @param segmentProperties
    * @param dataType                     DataType
    * @return the generator instance
    */
   public static RowLevelFilterExecuterImpl getRowLevelRangeTypeExecuter(
-      FilterExecuterType filterExecuterType, FilterResolverIntf filterExpressionResolverTree) {
+      FilterExecuterType filterExecuterType, FilterResolverIntf filterExpressionResolverTree,
+      SegmentProperties segmentProperties) {
     switch (filterExecuterType) {
 
       case ROWLEVEL_LESSTHAN:
@@ -49,7 +52,7 @@ public class RowLevelRangeTypeExecuterFacory {
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
+                .getFilterRangeValues(segmentProperties), segmentProperties);
       case ROWLEVEL_LESSTHAN_EQUALTO:
         return new RowLevelRangeLessThanEqualFilterExecuterImpl(
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
@@ -59,7 +62,7 @@ public class RowLevelRangeTypeExecuterFacory {
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
+                .getFilterRangeValues(segmentProperties), segmentProperties);
       case ROWLEVEL_GREATERTHAN_EQUALTO:
         return new RowLevelRangeGrtrThanEquaToFilterExecuterImpl(
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
@@ -69,7 +72,7 @@ public class RowLevelRangeTypeExecuterFacory {
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
+                .getFilterRangeValues(segmentProperties), segmentProperties);
       case ROWLEVEL_GREATERTHAN:
         return new RowLevelRangeGrtThanFiterExecuterImpl(
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
@@ -79,7 +82,7 @@ public class RowLevelRangeTypeExecuterFacory {
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getFilterExpression(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree).getTableIdentifier(),
             ((RowLevelRangeFilterResolverImpl) filterExpressionResolverTree)
-                .getFilterRangeValues());
+                .getFilterRangeValues(segmentProperties), segmentProperties);
       default:
         // Scenario wont come logic must break
         return null;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 1bebae2..442b17e 100644
--- a/core/src/main/java/org/carbondata/query/filter/resolver/AndFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/resolver/AndFilterResolverImpl.java
@@ -22,12 +22,13 @@ import java.util.SortedMap;
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbonfilterinterface.ExpressionType;
 
 public class AndFilterResolverImpl extends LogicalFilterResolverImpl {
 
   /**
-   *
+   *i
    */
   private static final long serialVersionUID = -761688076874662001L;
 
@@ -44,7 +45,7 @@ public class AndFilterResolverImpl extends LogicalFilterResolverImpl {
 
   @Override public void getEndKey(SegmentProperties segmentProperties,
       AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> noDicEndKeys) {
+      SortedMap<Integer, byte[]> noDicEndKeys) throws QueryExecutionException {
     leftEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
     rightEvalutor.getEndKey(segmentProperties, tableIdentifier, endKeys, noDicEndKeys);
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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 59ef1be..af17609 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
@@ -132,8 +132,7 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
       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)) {
+      if (!columnList.get(0).getDimension().hasEncoding(Encoding.DICTIONARY)) {
         dimColResolvedFilterInfo.populateFilterInfoBasedOnColumnType(
             FilterInfoTypeVisitorFactory.getResolvedFilterInfoVisitor(columnList.get(0)), metadata);
 
@@ -199,20 +198,16 @@ public class ConditionalFilterResolverImpl implements FilterResolverIntf {
    * method will get the start key based on the filter surrogates
    *
    * @return end IndexKey
+   * @throws QueryExecutionException
    */
   @Override public void getEndKey(SegmentProperties segmentProperties,
       AbsoluteTableIdentifier absoluteTableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
+      SortedMap<Integer, byte[]> setOfEndKeyByteArray) throws QueryExecutionException {
     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();
-      }
+      FilterUtil.getEndKey(dimColResolvedFilterInfo.getDimensionResolvedFilterInstance(),
+          absoluteTableIdentifier, endKeys, segmentProperties);
+      FilterUtil.getEndKeyForNoDictionaryDimension(dimColResolvedFilterInfo, segmentProperties,
+          setOfEndKeyByteArray);
     }
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index a8f1df5..4537ba6 100644
--- a/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
+++ b/core/src/main/java/org/carbondata/query/filter/resolver/FilterResolverIntf.java
@@ -23,6 +23,7 @@ import java.util.SortedMap;
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
 import org.carbondata.query.expression.Expression;
 import org.carbondata.query.expression.exception.FilterUnsupportedException;
@@ -81,9 +82,11 @@ public interface FilterResolverIntf extends Serializable {
    * @param setOfEndKeyByteArray
    * @param endKeys
    * @return
+   * @throws QueryExecutionException
    */
   void getEndKey(SegmentProperties segmentProperties, AbsoluteTableIdentifier tableIdentifier,
-      long[] endKeys, SortedMap<Integer, byte[]> setOfEndKeyByteArray);
+      long[] endKeys, SortedMap<Integer, byte[]> setOfEndKeyByteArray)
+      throws QueryExecutionException;
 
   /**
    * API will return the filter executer type which will be used to evaluate

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 8bf1395..46b03e4 100644
--- a/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/resolver/LogicalFilterResolverImpl.java
@@ -22,6 +22,7 @@ import java.util.SortedMap;
 
 import org.carbondata.core.carbon.AbsoluteTableIdentifier;
 import org.carbondata.core.carbon.datastore.block.SegmentProperties;
+import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbonfilterinterface.ExpressionType;
 import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
 import org.carbondata.query.expression.Expression;
@@ -87,7 +88,7 @@ public class LogicalFilterResolverImpl implements FilterResolverIntf {
 
   @Override public void getEndKey(SegmentProperties segmentProperties,
       AbsoluteTableIdentifier tableIdentifier, long[] endKeys,
-      SortedMap<Integer, byte[]> setOfEndKeyByteArray) {
+      SortedMap<Integer, byte[]> setOfEndKeyByteArray) throws QueryExecutionException {
   }
 
   @Override public FilterExecuterType getFilterExecuterType() {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index a9add19..edb4ece 100644
--- a/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
+++ b/core/src/main/java/org/carbondata/query/filter/resolver/RowLevelRangeFilterResolverImpl.java
@@ -28,8 +28,11 @@ import org.carbondata.common.logging.LogService;
 import org.carbondata.common.logging.LogServiceFactory;
 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.CarbonMeasure;
 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.ByteUtil;
 import org.carbondata.query.carbon.executor.exception.QueryExecutionException;
 import org.carbondata.query.carbonfilterinterface.FilterExecuterType;
@@ -38,6 +41,7 @@ import org.carbondata.query.expression.Expression;
 import org.carbondata.query.expression.ExpressionResult;
 import org.carbondata.query.expression.conditional.BinaryConditionalExpression;
 import org.carbondata.query.expression.exception.FilterIllegalMemberException;
+import org.carbondata.query.expression.exception.FilterUnsupportedException;
 import org.carbondata.query.expression.logical.BinaryLogicalExpression;
 import org.carbondata.query.filter.resolver.resolverinfo.DimColumnResolvedFilterInfo;
 import org.carbondata.query.filter.resolver.resolverinfo.MeasureColumnResolvedFilterInfo;
@@ -67,19 +71,25 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
   }
 
   /**
-   * This method will return the filter values which is present in the range leve
+   * This method will return the filter values which is present in the range level
    * conditional expressions.
    *
    * @return
    */
-  public byte[][] getFilterRangeValues() {
-    List<byte[]> filterValuesList = new ArrayList<byte[]>();
-    if (null != dimColEvaluatorInfoList.get(0).getFilterValues()) {
-      filterValuesList =
+  public byte[][] getFilterRangeValues(SegmentProperties segmentProperties) {
+
+    if (null != dimColEvaluatorInfoList.get(0).getFilterValues() && !dimColEvaluatorInfoList.get(0)
+        .getDimension().hasEncoding(Encoding.DICTIONARY)) {
+      List<byte[]> noDictFilterValuesList =
           dimColEvaluatorInfoList.get(0).getFilterValues().getNoDictionaryFilterValuesList();
-      return filterValuesList.toArray((new byte[filterValuesList.size()][]));
+      return noDictFilterValuesList.toArray((new byte[noDictFilterValuesList.size()][]));
+    } else if (null != dimColEvaluatorInfoList.get(0).getFilterValues() && dimColEvaluatorInfoList
+        .get(0).getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+      return FilterUtil.getKeyArray(this.dimColEvaluatorInfoList.get(0).getFilterValues(),
+          this.dimColEvaluatorInfoList.get(0).getDimension(),
+          segmentProperties.getDimensionKeyGenerator());
     }
-    return filterValuesList.toArray((new byte[filterValuesList.size()][]));
+    return null;
 
   }
 
@@ -91,6 +101,7 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
   public void getStartKey(SegmentProperties segmentProperties, long[] startKey,
       SortedMap<Integer, byte[]> noDictStartKeys) {
     if (null == dimColEvaluatorInfoList.get(0).getStarIndexKey()) {
+      FilterUtil.getStartKey(dimColEvaluatorInfoList.get(0), segmentProperties, startKey);
       FilterUtil
           .getStartKeyForNoDictionaryDimension(dimColEvaluatorInfoList.get(0), segmentProperties,
               noDictStartKeys);
@@ -171,7 +182,11 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
           dimColumnEvaluatorInfo.setRowIndex(index++);
           dimColumnEvaluatorInfo.setDimension(columnExpression.getDimension());
           dimColumnEvaluatorInfo.setDimensionExistsInCurrentSilce(false);
-          filterInfo.setFilterListForNoDictionaryCols(getNoDictionaryRangeValues());
+          if (columnExpression.getDimension().hasEncoding(Encoding.DIRECT_DICTIONARY)) {
+            filterInfo.setFilterList(getDirectSurrogateValues(columnExpression));
+          } else {
+            filterInfo.setFilterListForNoDictionaryCols(getNoDictionaryRangeValues());
+          }
           filterInfo.setIncludeFilter(isIncludeFilter);
           dimColumnEvaluatorInfo.setFilterValues(filterInfo);
           dimColumnEvaluatorInfo
@@ -191,6 +206,30 @@ public class RowLevelRangeFilterResolverImpl extends ConditionalFilterResolverIm
     }
   }
 
+  private List<Integer> getDirectSurrogateValues(ColumnExpression columnExpression) {
+    List<ExpressionResult> listOfExpressionResults = new ArrayList<ExpressionResult>(20);
+    DirectDictionaryGenerator directDictionaryGenerator = DirectDictionaryKeyGeneratorFactory
+        .getDirectDictionaryGenerator(columnExpression.getDimension().getDataType());
+
+    if (this.getFilterExpression() instanceof BinaryConditionalExpression) {
+      listOfExpressionResults =
+          ((BinaryConditionalExpression) this.getFilterExpression()).getLiterals();
+    }
+    List<Integer> filterValuesList = new ArrayList<Integer>(20);
+    try {
+      // if any filter member provided by user is invalid throw error else
+      // system can display inconsistent result.
+      for (ExpressionResult result : listOfExpressionResults) {
+        filterValuesList.add(directDictionaryGenerator
+            .generateDirectSurrogateKey(result.getString(),
+                CarbonCommonConstants.CARBON_TIMESTAMP_DEFAULT_FORMAT));
+      }
+    } catch (FilterIllegalMemberException e) {
+      new FilterUnsupportedException(e);
+    }
+    return filterValuesList;
+  }
+
   /**
    * Method will return the DimColumnResolvedFilterInfo instance which consists
    * the mapping of the respective dimension and its surrogates involved in

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index eea27e9..dded529 100644
--- a/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java
+++ b/core/src/main/java/org/carbondata/query/filters/FilterExpressionProcessor.java
@@ -123,7 +123,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
     DataRefNode startBlock = blockFinder.findFirstDataBlock(btreeNode, searchStartKey);
     DataRefNode endBlock = blockFinder.findLastDataBlock(btreeNode, searchEndKey);
     FilterExecuter filterExecuter =
-            FilterUtil.getFilterExecuterTree(filterResolver, tableSegment.getSegmentProperties());
+        FilterUtil.getFilterExecuterTree(filterResolver, tableSegment.getSegmentProperties());
     while (startBlock != endBlock) {
       addBlockBasedOnMinMaxValue(filterExecuter, listOfDataBlocksToScan, startBlock,
           tableSegment.getSegmentProperties());
@@ -185,6 +185,7 @@ public class FilterExpressionProcessor implements FilterProcessor {
    *
    * @param filterResolverTree
    * @param tableIdentifier
+   * @throws FilterUnsupportedException
    * @throws QueryExecutionException
    */
   private void traverseAndResolveTree(FilterResolverIntf filterResolverTree,
@@ -285,11 +286,6 @@ public class FilterExpressionProcessor implements FilterProcessor {
                 == 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);
             }
@@ -308,10 +304,9 @@ public class FilterExpressionProcessor implements FilterProcessor {
           if (!currentCondExpression.getColumnList().get(0).getCarbonColumn()
               .hasEncoding(Encoding.DICTIONARY)) {
             if (FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getLeft())
-                && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight())
-                || (FilterUtil
-                    .checkIfExpressionContainsUnknownExp(currentCondExpression.getRight())
-                || FilterUtil
+                && FilterUtil.checkIfExpressionContainsColumn(currentCondExpression.getRight()) || (
+                FilterUtil.checkIfExpressionContainsUnknownExp(currentCondExpression.getRight())
+                    || FilterUtil
                     .checkIfExpressionContainsUnknownExp(currentCondExpression.getLeft()))) {
               return new RowLevelFilterResolverImpl(expression, isExpressionResolve, false,
                   tableIdentifier);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/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
index 810cbaf..ee72f62 100644
--- 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
@@ -133,7 +133,8 @@ public final class FilterUtil {
         case ROWLEVEL_GREATERTHAN_EQUALTO:
         case ROWLEVEL_GREATERTHAN:
           return RowLevelRangeTypeExecuterFacory
-              .getRowLevelRangeTypeExecuter(filterExecuterType, filterExpressionResolverTree);
+              .getRowLevelRangeTypeExecuter(filterExecuterType, filterExpressionResolverTree,
+                  segmentProperties);
         case ROWLEVEL:
         default:
           return new RowLevelFilterExecuterImpl(

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/integration/spark/src/test/resources/data2.csv
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/resources/data2.csv b/integration/spark/src/test/resources/data2.csv
index 6187ac3..2e9200d 100644
--- a/integration/spark/src/test/resources/data2.csv
+++ b/integration/spark/src/test/resources/data2.csv
@@ -1,4 +1,4 @@
 ID,date,country,name,phonetype,serialname,salary
-4,21-01-2014,china,aaa4,phone2435,ASD66902,15003
-abc,,china,aaa5,phone2441,ASD90633,15004
-6,21-01-2014,china,aaa6,phone294,ASD59961,15005
+4,2014-01-21 00:00:00,china,aaa4,phone2435,ASD66902,15003
+abc,2014-01-22 00:00:00,china,aaa5,phone2441,ASD90633,15004
+6,2014-03-07 00:00:00,china,aaa6,phone294,ASD59961,15005

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
index 44195d2..65e125e 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/detailquery/HighCardinalityDataTypesTestCase.scala
@@ -23,6 +23,8 @@ 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
+import org.carbondata.core.constants.CarbonCommonConstants
+import org.carbondata.core.util.CarbonProperties
 
 
 /**
@@ -76,6 +78,8 @@ class NO_DICTIONARY_COL_TestCase extends QueryTest with BeforeAndAfterAll {
       "name String, phonetype String, serialname String, salary Int) " +
         "STORED BY 'org.apache.carbondata.format' " +  "TBLPROPERTIES('DICTIONARY_EXCLUDE'='ID')"
     )
+        CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
     sql(
       s"LOAD DATA LOCAL INPATH './src/test/resources/data2.csv' INTO TABLE filtertestTable OPTIONS"+
         s"('DELIMITER'= ',', " +

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/462eb393/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
index afbda24..c272e2a 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
@@ -40,38 +40,31 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists filtertestTables")
     sql("drop table if exists filtertestTablesWithDecimal")
     sql("drop table if exists filtertestTablesWithNull")
-    sql("drop table if exists filterWithTimeStamp")
+    sql("drop table if exists filterTimestampDataType")
     sql("drop table if exists noloadtable")
     sql("CREATE TABLE filtertestTables (ID int, date Timestamp, country String, " +
       "name String, phonetype String, serialname String, salary int) " +
-      "STORED BY 'org.apache.carbondata.format'"
+        "STORED BY 'org.apache.carbondata.format'"
     )
     sql("CREATE TABLE noloadtable (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, "MM-dd-yyyy HH:mm:ss")
-
-    sql("CREATE TABLE filterWithTimeStamp (ID int, date Timestamp, country String, " +
+     CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "MM-dd-yyyy HH:mm:ss")
+        
+     sql("CREATE TABLE filterTimestampDataType (ID int, date Timestamp, country String, " +
       "name String, phonetype String, serialname String, salary int) " +
-      "STORED BY 'org.apache.carbondata.format'"
+        "STORED BY 'org.apache.carbondata.format'"
     )
+       CarbonProperties.getInstance()
+        .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "MM-dd-yyyy HH:mm:ss")
     sql(
       s"LOAD DATA LOCAL INPATH './src/test/resources/data2_DiffTimeFormat.csv' INTO TABLE " +
-        s"filterWithTimeStamp " +
+        s"filterTimestampDataType " +
         s"OPTIONS('DELIMITER'= ',', " +
         s"'FILEHEADER'= '')"
     )
-
-    test("Time stamp filter with diff time format for load ") {
-      checkAnswer(
-        sql("select date  from filterWithTimeStamp where date > '2014-07-10 00:00:00'"),
-        Seq(Row(Timestamp.valueOf("2014-07-20 00:00:00.0")),
-          Row(Timestamp.valueOf("2014-07-25 00:00:00.0"))
-        )
-      )
-    }
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
     sql(
@@ -83,7 +76,7 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
       "CREATE TABLE filtertestTablesWithDecimal (ID decimal, date Timestamp, country " +
         "String, " +
         "name String, phonetype String, serialname String, salary int) " +
-        "STORED BY 'org.apache.carbondata.format'"
+      "STORED BY 'org.apache.carbondata.format'"
     )
     sql(
       s"LOAD DATA LOCAL INPATH './src/test/resources/dataDiff.csv' INTO TABLE " +
@@ -95,10 +88,10 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
       "CREATE TABLE filtertestTablesWithNull (ID int, date Timestamp, country " +
         "String, " +
         "name String, phonetype String, serialname String,salary int) " +
-        "STORED BY 'org.apache.carbondata.format'"
+      "STORED BY 'org.apache.carbondata.format'"
     )
     CarbonProperties.getInstance()
-      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy-MM-dd HH:mm:ss")
     sql(
       s"LOAD DATA LOCAL INPATH './src/test/resources/data2.csv' INTO TABLE " +
         s"filtertestTablesWithNull " +
@@ -107,14 +100,21 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     )
   }
 
-
+    
   test("Is not null filter") {
     checkAnswer(
       sql("select id from filtertestTablesWithNull " + "where id is not null"),
       Seq(Row(4), Row(6))
     )
   }
-  test("Multi column with invalid member filter") {
+  
+    test("Between  filter") {
+    checkAnswer(
+      sql("select date from filtertestTablesWithNull " + " where date between '2014-01-20 00:00:00' and '2014-01-28 00:00:00'"),
+      Seq(Row(Timestamp.valueOf("2014-01-21 00:00:00")), Row(Timestamp.valueOf("2014-01-22 00:00:00")))
+    )
+  }
+    test("Multi column with invalid member filter") {
     checkAnswer(
       sql("select id from filtertestTablesWithNull " + "where id = salary"),
       Seq()
@@ -168,7 +168,7 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
       Seq(Row(1, "china"))
     )
   }
-
+  
   test("filter query over table having no data") {
     checkAnswer(
       sql("select * from noloadtable " + "where country='china' and name='aaa1'"),
@@ -176,6 +176,38 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     )
   }
 
+
+    
+     test("Time stamp filter with diff time format for load greater") {
+    checkAnswer(
+      sql("select date  from filterTimestampDataType where date > '2014-07-10 00:00:00'"),
+      Seq(Row(Timestamp.valueOf("2014-07-20 00:00:00.0")),
+        Row(Timestamp.valueOf("2014-07-25 00:00:00.0"))
+      )
+    )
+  }
+    test("Time stamp filter with diff time format for load less") {
+    checkAnswer(
+      sql("select date  from filterTimestampDataType where date < '2014-07-20 00:00:00'"),
+      Seq(Row(Timestamp.valueOf("2014-07-10 00:00:00.0"))
+      )
+    )
+  }
+   test("Time stamp filter with diff time format for load less than equal") {
+    checkAnswer(
+      sql("select date  from filterTimestampDataType where date <= '2014-07-20 00:00:00'"),
+      Seq(Row(Timestamp.valueOf("2014-07-10 00:00:00.0")),Row(Timestamp.valueOf("2014-07-20 00:00:00.0"))
+      )
+    )
+  }
+      test("Time stamp filter with diff time format for load greater than equal") {
+    checkAnswer(
+      sql("select date  from filterTimestampDataType where date >= '2014-07-20 00:00:00'"),
+      Seq(Row(Timestamp.valueOf("2014-07-20 00:00:00.0")),Row(Timestamp.valueOf("2014-07-25 00:00:00.0"))
+      )
+    )
+  }
+
   override def afterAll {
     sql("drop table noloadtable")
     CarbonProperties.getInstance()


[35/50] [abbrv] incubator-carbondata git commit: [Bug] Finding the merge folder name using int. (#772)

Posted by ch...@apache.org.
[Bug] Finding the merge folder name using int. (#772)



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

Branch: refs/heads/master
Commit: 114b4717995c7e7f0812516ee054737d459f79a9
Parents: b5fc518
Author: ravikiran23 <ra...@gmail.com>
Authored: Wed Jun 29 23:57:38 2016 +0530
Committer: Venkata Ramana G <g....@gmail.com>
Committed: Wed Jun 29 23:57:38 2016 +0530

----------------------------------------------------------------------
 .../spark/merger/CarbonDataMergerUtil.java           | 15 +++++++++------
 1 file changed, 9 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/114b4717/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
index 52e8a08..3aa66c2 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/merger/CarbonDataMergerUtil.java
@@ -113,14 +113,17 @@ public final class CarbonDataMergerUtil {
    */
   public static String getMergedLoadName(List<LoadMetadataDetails> segmentsToBeMergedList) {
     String firstSegmentName = segmentsToBeMergedList.get(0).getLoadName();
-    // check if segment is already merged or not.
-    if (null != segmentsToBeMergedList.get(0).getMergedLoadName()) {
-      firstSegmentName = segmentsToBeMergedList.get(0).getMergedLoadName();
+    if (firstSegmentName.contains(".")) {
+      String beforeDecimal = firstSegmentName.substring(0, firstSegmentName.indexOf("."));
+      String afterDecimal = firstSegmentName.substring(firstSegmentName.indexOf(".") + 1);
+      int fraction = Integer.parseInt(afterDecimal) + 1;
+      String mergedSegmentName = beforeDecimal + "." + fraction;
+      return CarbonCommonConstants.LOAD_FOLDER + mergedSegmentName;
+    } else {
+      String mergeName = firstSegmentName + "." + 1;
+      return CarbonCommonConstants.LOAD_FOLDER + mergeName;
     }
 
-    float segmentNumber = Float.parseFloat(firstSegmentName);
-    segmentNumber += 0.1;
-    return CarbonCommonConstants.LOAD_FOLDER + segmentNumber;
   }
 
   public static boolean updateLoadMetadataWithMergeStatus(List<LoadMetadataDetails> loadsToMerge,


[24/50] [abbrv] incubator-carbondata git commit: [issue-CARBONDATA-15] filter query throwing error if the query applied over a table having no data. (#760)

Posted by ch...@apache.org.
[issue-CARBONDATA-15] filter query throwing error if the query applied over a table having no data. (#760)



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

Branch: refs/heads/master
Commit: e88529f0f169cbf3b96728dad5594e396eb84045
Parents: 38d84e0
Author: Mohammad Shahid Khan <mo...@gmail.com>
Authored: Mon Jun 27 19:40:24 2016 +0530
Committer: Vimal-Das <vi...@gmail.com>
Committed: Mon Jun 27 07:10:24 2016 -0700

----------------------------------------------------------------------
 .../carbondata/spark/rdd/CarbonQueryRDD.scala   | 13 +++--
 .../filterexpr/FilterProcessorTestCase.scala    | 57 ++++++++++++--------
 2 files changed, 42 insertions(+), 28 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e88529f0/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
index 60c6a78..f0a8706 100644
--- a/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonQueryRDD.scala
+++ b/integration/spark/src/main/scala/org/carbondata/spark/rdd/CarbonQueryRDD.scala
@@ -80,11 +80,14 @@ class CarbonQueryRDD[K, V](
     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)
+      // before applying filter check whether segments are available in the table.
+      val splits = carbonInputFormat.getSplits(job)
+      if (!splits.isEmpty) {
+        var filterResolver = carbonInputFormat
+          .getResolvedFilter(job.getConfiguration, filterExpression)
+        CarbonInputFormat.setFilterPredicates(job.getConfiguration, filterResolver)
+        queryModel.setFilterExpressionResolverTree(filterResolver)
+      }
     }
     catch {
       case e: Exception =>

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e88529f0/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
index 41930df..cd34c97 100644
--- a/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
+++ b/integration/spark/src/test/scala/org/carbondata/spark/testsuite/filterexpr/FilterProcessorTestCase.scala
@@ -19,14 +19,14 @@
 
 package org.carbondata.spark.testsuite.filterexpr
 
-import org.apache.spark.sql.common.util.CarbonHiveContext._
-import org.apache.spark.sql.common.util.QueryTest
-import org.apache.spark.sql.Row
-import org.scalatest.BeforeAndAfterAll
 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.carbondata.core.constants.CarbonCommonConstants
 import org.carbondata.core.util.CarbonProperties
+import org.scalatest.BeforeAndAfterAll
 
 /**
   * Test Class for filter expression query on String datatypes
@@ -41,17 +41,21 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     sql("drop table if exists filtertestTablesWithDecimal")
     sql("drop table if exists filtertestTablesWithNull")
     sql("drop table if exists filterWithTimeStamp")
+    sql("drop table if exists noloadtable")
     sql("CREATE TABLE filtertestTables (ID int, date Timestamp, country String, " +
       "name String, phonetype String, serialname String, salary int) " +
-        "STORED BY 'org.apache.carbondata.format'"
+      "STORED BY 'org.apache.carbondata.format'"
+    )
+    sql("CREATE TABLE noloadtable (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, "MM-dd-yyyy HH:mm:ss")
-        
+    CarbonProperties.getInstance()
+      .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "MM-dd-yyyy HH:mm:ss")
+
     sql("CREATE TABLE filterWithTimeStamp (ID int, date Timestamp, country String, " +
       "name String, phonetype String, serialname String, salary int) " +
-        "STORED BY 'org.apache.carbondata.format'"
+      "STORED BY 'org.apache.carbondata.format'"
     )
     sql(
       s"LOAD DATA LOCAL INPATH './src/test/resources/data2_DiffTimeFormat.csv' INTO TABLE " +
@@ -59,15 +63,15 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
         s"OPTIONS('DELIMITER'= ',', " +
         s"'FILEHEADER'= '')"
     )
-    
-     test("Time stamp filter with diff time format for load ") {
-    checkAnswer(
-      sql("select date  from filterWithTimeStamp where date > '2014-07-10 00:00:00'"),
-      Seq(Row(Timestamp.valueOf("2014-07-20 00:00:00.0")),
-        Row(Timestamp.valueOf("2014-07-25 00:00:00.0"))
+
+    test("Time stamp filter with diff time format for load ") {
+      checkAnswer(
+        sql("select date  from filterWithTimeStamp where date > '2014-07-10 00:00:00'"),
+        Seq(Row(Timestamp.valueOf("2014-07-20 00:00:00.0")),
+          Row(Timestamp.valueOf("2014-07-25 00:00:00.0"))
+        )
       )
-    )
-  }
+    }
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
     sql(
@@ -79,7 +83,7 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
       "CREATE TABLE filtertestTablesWithDecimal (ID decimal, date Timestamp, country " +
         "String, " +
         "name String, phonetype String, serialname String, salary int) " +
-      "STORED BY 'org.apache.carbondata.format'"
+        "STORED BY 'org.apache.carbondata.format'"
     )
     sql(
       s"LOAD DATA LOCAL INPATH './src/test/resources/dataDiff.csv' INTO TABLE " +
@@ -91,7 +95,7 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
       "CREATE TABLE filtertestTablesWithNull (ID int, date Timestamp, country " +
         "String, " +
         "name String, phonetype String, serialname String,salary int) " +
-      "STORED BY 'org.apache.carbondata.format'"
+        "STORED BY 'org.apache.carbondata.format'"
     )
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
@@ -103,14 +107,14 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     )
   }
 
-    
+
   test("Is not null filter") {
     checkAnswer(
       sql("select id from filtertestTablesWithNull " + "where id is not null"),
       Seq(Row(4), Row(6))
     )
   }
-    test("Multi column with invalid member filter") {
+  test("Multi column with invalid member filter") {
     checkAnswer(
       sql("select id from filtertestTablesWithNull " + "where id = salary"),
       Seq()
@@ -165,8 +169,15 @@ class FilterProcessorTestCase extends QueryTest with BeforeAndAfterAll {
     )
   }
 
+  test("filter query over table having no data") {
+    checkAnswer(
+      sql("select * from noloadtable " + "where country='china' and name='aaa1'"),
+      Seq()
+    )
+  }
+
   override def afterAll {
-    // sql("drop cube filtertestTable")
+    sql("drop cube noloadtable")
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "dd-MM-yyyy")
   }